diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 0ed8de48530..a1a686cbf4a 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -296,9 +296,9 @@ dependencies = [ [[package]] name = "async-trait" -version = "0.1.72" +version = "0.1.73" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6dde6e4ed435a4c1ee4e73592f5ba9da2151af10076cc04858746af9352d09" +checksum = "bc00ceb34980c03614e35a3a4e218276a0a824e911d07651cd0d858a51e8c0f0" dependencies = [ "proc-macro2", "quote", @@ -2777,9 +2777,9 @@ checksum = "d897f394bad6a705d5f4104762e116a75639e470d80901eed05a860a95cb1904" [[package]] name = "httpdate" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4a1e36c821dbe04574f602848a19f742f4fb3c98d40449f11bcad18d6b17421" +checksum = "df3b46402a9d5adb4c86a0cf463f42e19994e3ee891101b1841f30a545cb49a9" [[package]] name = "humantime" @@ -3439,9 +3439,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.19" +version = "0.4.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b06a4cde4c0f271a446782e3eff8de789548ce57dbc8eca9292c27f4a42004b4" +checksum = "b5e6163cb8c49088c2c36f57875e58ccd8c87c7427f7fbd50ea6710b2f3f2e8f" [[package]] name = "loom" @@ -5329,7 +5329,6 @@ dependencies = [ "mrecordlog", "once_cell", "prost", - "prost-build", "quickwit-actors", "quickwit-codegen", "quickwit-common", @@ -5539,6 +5538,7 @@ version = "0.6.3" dependencies = [ "anyhow", "async-trait", + "bytes", "dyn-clone", "glob", "http", @@ -7545,9 +7545,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.30.0" +version = "1.31.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d3ce25f50619af8b0aec2eb23deebe84249e19e2ddd393a6e16e3300a6dadfd" +checksum = "40de3a2ba249dcb097e01be5e67a5ff53cf250397715a071a81543e8a832a920" dependencies = [ "backtrace", "bytes", @@ -8791,9 +8791,9 @@ checksum = "1a515f5799fe4961cb532f983ce2b23082366b898e52ffbce459c86f67c8378a" [[package]] name = "winnow" -version = "0.5.7" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19f495880723d0999eb3500a9064d8dbcf836460b24c17df80ea7b5794053aac" +checksum = "5504cc7644f4b593cbc05c4a55bf9bd4e94b867c3c0bd440934174d50482427d" dependencies = [ "memchr", ] diff --git a/quickwit/quickwit-cli/src/index.rs b/quickwit/quickwit-cli/src/index.rs index 8c7b4af957c..db9cfdf655e 100644 --- a/quickwit/quickwit-cli/src/index.rs +++ b/quickwit/quickwit-cli/src/index.rs @@ -39,7 +39,7 @@ use quickwit_config::{ConfigFormat, IndexConfig}; use quickwit_indexing::models::IndexingStatistics; use quickwit_indexing::IndexingPipeline; use quickwit_metastore::{IndexMetadata, Split, SplitState}; -use quickwit_proto::{SortField, SortOrder}; +use quickwit_proto::search::{SortField, SortOrder}; use quickwit_rest_client::models::IngestSource; use quickwit_rest_client::rest_client::{CommitType, IngestEvent}; use quickwit_search::SearchResponseRest; diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 39471992488..8a0daabeff2 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -48,7 +48,7 @@ use quickwit_indexing::models::{ }; use quickwit_indexing::IndexingPipeline; use quickwit_metastore::Metastore; -use quickwit_proto::SearchResponse; +use quickwit_proto::search::SearchResponse; use quickwit_search::{single_node_search, SearchResponseRest}; use quickwit_serve::{ search_request_from_api_request, BodyFormat, SearchRequestQueryString, SortBy, diff --git a/quickwit/quickwit-codegen/src/codegen.rs b/quickwit/quickwit-codegen/src/codegen.rs index 0b599454567..300e157d457 100644 --- a/quickwit/quickwit-codegen/src/codegen.rs +++ b/quickwit/quickwit-codegen/src/codegen.rs @@ -23,6 +23,8 @@ use prost_build::{Method, Service, ServiceGenerator}; use quote::{quote, ToTokens}; use syn::Ident; +use crate::ProstConfig; + pub struct Codegen; impl Codegen { @@ -39,7 +41,7 @@ impl Codegen { result_type_path, error_type_path, includes, - prost_build::Config::default(), + ProstConfig::default(), ) } @@ -49,7 +51,7 @@ impl Codegen { result_type_path: &str, error_type_path: &str, includes: &[&str], - mut prost_config: prost_build::Config, + mut prost_config: ProstConfig, ) -> anyhow::Result<()> { let service_generator = Box::new(QuickwitServiceGenerator::new( result_type_path, diff --git a/quickwit/quickwit-codegen/src/lib.rs b/quickwit/quickwit-codegen/src/lib.rs index c97d7434e9f..57b371deada 100644 --- a/quickwit/quickwit-codegen/src/lib.rs +++ b/quickwit/quickwit-codegen/src/lib.rs @@ -20,3 +20,4 @@ mod codegen; pub use codegen::Codegen; +pub use prost_build::Config as ProstConfig; diff --git a/quickwit/quickwit-common/src/stream_utils.rs b/quickwit/quickwit-common/src/stream_utils.rs index 70d46f3b0ae..aa89b483621 100644 --- a/quickwit/quickwit-common/src/stream_utils.rs +++ b/quickwit/quickwit-common/src/stream_utils.rs @@ -21,7 +21,7 @@ use std::any::TypeId; use std::fmt; use std::pin::Pin; -use futures::{Stream, StreamExt, TryStreamExt}; +use futures::{stream, Stream, TryStreamExt}; use tokio::sync::mpsc; use tokio_stream::wrappers::{ReceiverStream, UnboundedReceiverStream}; use tracing::warn; @@ -123,19 +123,20 @@ impl From> for ServiceStream where T: Send + 'static { fn from(streaming: tonic::Streaming) -> Self { - let ok_streaming = streaming.filter_map(|message| { - Box::pin(async move { - message - .map_err(|status| { - warn!(status=?status, "gRPC transport error."); - status - }) - .ok() + let message_stream = stream::unfold(streaming, |mut streaming| { + Box::pin(async { + match streaming.message().await { + Ok(Some(message)) => Some((message, streaming)), + Ok(None) => None, + Err(error) => { + warn!(error=?error, "gRPC transport error."); + None + } + } }) }); - Self { - inner: Box::pin(ok_streaming), + inner: Box::pin(message_stream), } } } diff --git a/quickwit/quickwit-common/src/tower/mod.rs b/quickwit/quickwit-common/src/tower/mod.rs index ab8d887bcb5..79d5dd5e0e9 100644 --- a/quickwit/quickwit-common/src/tower/mod.rs +++ b/quickwit/quickwit-common/src/tower/mod.rs @@ -37,7 +37,7 @@ pub use box_service::BoxService; pub use buffer::{Buffer, BufferError, BufferLayer}; pub use change::Change; pub use estimate_rate::{EstimateRate, EstimateRateLayer}; -use futures::{Future, Stream}; +use futures::Future; pub use metrics::{PrometheusMetrics, PrometheusMetricsLayer}; pub use pool::Pool; pub use rate::{ConstantRate, Rate}; @@ -51,8 +51,6 @@ pub type BoxFuture = Pin> + Send + 's pub type BoxFutureInfaillible = Pin + Send + 'static>>; -pub type BoxStream = Pin + Send + Unpin + 'static>>; - pub trait Cost { fn cost(&self) -> u64; } diff --git a/quickwit/quickwit-common/src/tower/transport.rs b/quickwit/quickwit-common/src/tower/transport.rs index d509d8b568c..83076a7a8c4 100644 --- a/quickwit/quickwit-common/src/tower/transport.rs +++ b/quickwit/quickwit-common/src/tower/transport.rs @@ -38,7 +38,8 @@ use tower::discover::Change as TowerChange; use tower::load::{CompleteOnResponse, PendingRequestsDiscover}; use tower::{BoxError, Service, ServiceExt}; -use super::{BoxFuture, BoxStream, Change}; +use super::{BoxFuture, Change}; +use crate::BoxStream; // Transforms a boxed stream of `Change` into a stream of `Result, Infallible>>` while keeping track of the number of connections. diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index fb57a418d3e..46fb386ebf9 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -20,7 +20,10 @@ use anyhow::Context; use async_trait::async_trait; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox}; -use quickwit_proto::control_plane::{NotifyIndexChangeRequest, NotifyIndexChangeResponse}; +use quickwit_proto::control_plane::{ + CloseShardsRequest, CloseShardsResponse, ControlPlaneResult, GetOpenShardsRequest, + GetOpenShardsResponse, NotifyIndexChangeRequest, NotifyIndexChangeResponse, +}; use tracing::debug; use crate::scheduler::IndexingScheduler; @@ -51,7 +54,7 @@ impl ControlPlane { #[async_trait] impl Handler for ControlPlane { - type Reply = quickwit_proto::control_plane::Result; + type Reply = ControlPlaneResult; async fn handle( &mut self, @@ -66,3 +69,29 @@ impl Handler for ControlPlane { Ok(Ok(NotifyIndexChangeResponse {})) } } + +#[async_trait] +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + _request: GetOpenShardsRequest, + _: &ActorContext, + ) -> Result { + unimplemented!() + } +} + +#[async_trait] +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + _request: CloseShardsRequest, + _: &ActorContext, + ) -> Result { + unimplemented!() + } +} diff --git a/quickwit/quickwit-control-plane/src/scheduler.rs b/quickwit/quickwit-control-plane/src/scheduler.rs index d9adf1ee89e..2a085314631 100644 --- a/quickwit/quickwit-control-plane/src/scheduler.rs +++ b/quickwit/quickwit-control-plane/src/scheduler.rs @@ -29,7 +29,9 @@ use itertools::Itertools; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler}; use quickwit_config::SourceConfig; use quickwit_metastore::Metastore; -use quickwit_proto::control_plane::{NotifyIndexChangeRequest, NotifyIndexChangeResponse}; +use quickwit_proto::control_plane::{ + ControlPlaneResult, NotifyIndexChangeRequest, NotifyIndexChangeResponse, +}; use quickwit_proto::indexing::{ApplyIndexingPlanRequest, IndexingService, IndexingTask}; use serde::Serialize; use tracing::{debug, error, info, warn}; @@ -298,7 +300,7 @@ impl IndexingScheduler { #[async_trait] impl Handler for IndexingScheduler { - type Reply = quickwit_proto::control_plane::Result; + type Reply = ControlPlaneResult; async fn handle( &mut self, diff --git a/quickwit/quickwit-ingest/Cargo.toml b/quickwit/quickwit-ingest/Cargo.toml index 42d3b6e7097..ec8807efbff 100644 --- a/quickwit/quickwit-ingest/Cargo.toml +++ b/quickwit/quickwit-ingest/Cargo.toml @@ -19,6 +19,7 @@ mockall = { workspace = true, optional = true } mrecordlog = { workspace = true } once_cell = { workspace = true } prost = { workspace = true } +rand = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } thiserror = { workspace = true } @@ -41,10 +42,11 @@ rand_distr = { workspace = true } tempfile = { workspace = true } quickwit-actors = { workspace = true, features = ["testsuite"] } +quickwit-common = { workspace = true, features = ["testsuite"] } +quickwit-proto = { workspace = true, features = ["testsuite"] } [build-dependencies] quickwit-codegen = { workspace = true } -prost-build = { workspace = true } [features] testsuite = ["mockall"] diff --git a/quickwit/quickwit-ingest/build.rs b/quickwit/quickwit-ingest/build.rs index 62180bf50c0..638a17e87f1 100644 --- a/quickwit/quickwit-ingest/build.rs +++ b/quickwit/quickwit-ingest/build.rs @@ -17,18 +17,20 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use quickwit_codegen::Codegen; +use quickwit_codegen::{Codegen, ProstConfig}; fn main() { - let mut config = prost_build::Config::default(); - config.bytes(["DocBatch.doc_buffer"]); + // Legacy ingest codegen + let mut prost_config = ProstConfig::default(); + prost_config.bytes(["DocBatch.doc_buffer"]); + Codegen::run_with_config( &["src/ingest_service.proto"], "src/codegen/", "crate::Result", "crate::IngestServiceError", &[], - config, + prost_config, ) .unwrap(); } diff --git a/quickwit/quickwit-ingest/src/doc_batch.rs b/quickwit/quickwit-ingest/src/doc_batch.rs index 24bdd7c7201..bef99ca9554 100644 --- a/quickwit/quickwit-ingest/src/doc_batch.rs +++ b/quickwit/quickwit-ingest/src/doc_batch.rs @@ -72,7 +72,7 @@ where T: Buf + Default } /// Copies the command to the end of bytes::BufMut while returning the number of bytes copied - pub fn write(self, buf: &mut impl BufMut) -> usize { + pub fn write(self, mut buf: impl BufMut) -> usize { let self_buf = self.into_buf(); let len = self_buf.remaining(); buf.put(self_buf); diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs new file mode 100644 index 00000000000..31f65b34b30 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -0,0 +1,676 @@ +// 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::borrow::Borrow; +use std::collections::hash_map::Entry; +use std::collections::HashMap; +use std::fmt; +use std::ops::RangeInclusive; +use std::sync::Arc; + +use futures::StreamExt; +use mrecordlog::MultiRecordLog; +use quickwit_common::ServiceStream; +use quickwit_proto::ingest::ingester::{FetchResponseV2, IngesterService, OpenFetchStreamRequest}; +use quickwit_proto::ingest::{IngestV2Error, IngestV2Result}; +use quickwit_proto::types::{queue_id, NodeId, QueueId, ShardId, SourceId}; +use quickwit_proto::IndexUid; +use tokio::sync::{mpsc, watch, RwLock}; +use tokio::task::JoinHandle; +use tracing::{debug, error, warn}; + +use super::ingester::ShardStatus; +use crate::{ClientId, DocBatchBuilderV2, IngesterPool}; + +/// A fetch task is responsible for waiting and pushing new records written to a shard's record log +/// into a channel named `fetch_response_tx`. +pub(super) struct FetchTask { + /// Uniquely identifies the consumer of the fetch task for logging and debugging purposes. + client_id: ClientId, + index_uid: IndexUid, + 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, + mrecordlog: 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, + batch_num_bytes: usize, +} + +impl fmt::Debug for FetchTask { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("FetchTask") + .field("client_id", &self.client_id) + .field("index_uid", &self.index_uid) + .field("source_id", &self.source_id) + .field("shard_id", &self.shard_id) + .finish() + } +} + +// TODO: Drop when `Iterator::advance_by()` is stabilized. +fn advance_by(range: &mut RangeInclusive, len: u64) { + *range = *range.start() + len..=*range.end(); +} + +impl FetchTask { + pub const DEFAULT_BATCH_NUM_BYTES: usize = 1024 * 1024; // 1 MiB + + pub fn spawn( + open_fetch_stream_request: OpenFetchStreamRequest, + mrecordlog: Arc>, + shard_status_rx: watch::Receiver, + batch_num_bytes: usize, + ) -> ServiceStream> { + 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 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, + mrecordlog, + fetch_response_tx, + shard_status_rx, + batch_num_bytes, + }; + let future = async move { fetch_task.run().await }; + tokio::spawn(future); + fetch_stream + } + + /// Waits for new records. Returns `false` if the ingester is dropped. + async fn wait_for_new_records(&mut self) -> bool { + while self.shard_status_rx.borrow().replication_position_inclusive + < *self.fetch_range.start() + { + if self.shard_status_rx.changed().await.is_err() { + // The ingester was dropped. + return false; + } + } + true + } + + async fn run(&mut self) { + debug!( + client_id=%self.client_id, + index_uid=%self.index_uid, + source_id=%self.source_id, + shard_id=%self.shard_id, + fetch_range=?self.fetch_range, + "Spawning fetch task." + ); + while !self.fetch_range.is_empty() { + if !self.wait_for_new_records().await { + break; + } + let fetch_range = self.fetch_range.clone(); + let mrecordlog_guard = self.mrecordlog.read().await; + + let Ok(docs) = mrecordlog_guard.range(&self.queue_id, fetch_range) else { + warn!( + client_id=%self.client_id, + index_uid=%self.index_uid, + source_id=%self.source_id, + shard_id=%self.shard_id, + "Failed to read from record log because it was dropped." + ); + break; + }; + let mut doc_batch_builder = DocBatchBuilderV2::with_capacity(self.batch_num_bytes); + + for (_position, doc) in docs { + if doc_batch_builder.num_bytes() + doc.len() > doc_batch_builder.capacity() { + break; + } + doc_batch_builder.add_doc(doc.borrow()); + } + // Drop the lock while we send the message. + drop(mrecordlog_guard); + + let doc_batch = doc_batch_builder.build(); + let num_docs = doc_batch.num_docs() as u64; + + let fetch_response = FetchResponseV2 { + index_uid: self.index_uid.clone().into(), + source_id: self.source_id.clone(), + shard_id: self.shard_id, + doc_batch: Some(doc_batch), + from_position_inclusive: *self.fetch_range.start(), + }; + advance_by(&mut self.fetch_range, num_docs); + + if self + .fetch_response_tx + .send(Ok(fetch_response)) + .await + .is_err() + { + // The ingester was dropped. + break; + } + } + debug!( + client_id=%self.client_id, + index_uid=%self.index_uid, + source_id=%self.source_id, + shard_id=%self.shard_id, + to_position_inclusive=%self.fetch_range.end(), + "Fetch task completed." + ) + } +} + +/// Combines multiple fetch streams originating from different ingesters into a single stream. It +/// tolerates the failure of ingesters and automatically fails over to replica shards. +pub struct MultiFetchStream { + self_node_id: NodeId, + client_id: ClientId, + ingester_pool: IngesterPool, + fetch_task_handles: HashMap>, + fetch_response_rx: mpsc::Receiver>, + fetch_response_tx: mpsc::Sender>, +} + +impl MultiFetchStream { + pub fn new(self_node_id: NodeId, client_id: ClientId, ingester_pool: IngesterPool) -> Self { + let (fetch_response_tx, fetch_response_rx) = mpsc::channel(3); + Self { + self_node_id, + client_id, + ingester_pool, + fetch_task_handles: HashMap::new(), + fetch_response_rx, + fetch_response_tx, + } + } + + /// Subscribes to a shard and fails over to the replica if an error occurs. + #[allow(clippy::too_many_arguments)] + pub async fn subscribe( + &mut self, + leader_id: NodeId, + follower_id_opt: Option, + index_uid: IndexUid, + source_id: SourceId, + shard_id: ShardId, + from_position_exclusive: Option, + to_position_inclusive: Option, + ) -> IngestV2Result<()> { + let queue_id = queue_id(index_uid.as_str(), &source_id, shard_id); + let entry = self.fetch_task_handles.entry(queue_id.clone()); + + if let Entry::Occupied(_) = entry { + return Err(IngestV2Error::Internal(format!( + "Stream has already subscribed to shard `{queue_id}`." + ))); + } + let (mut preferred_ingester_id, mut failover_ingester_id) = + 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).await 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 fetch_task_future = fetch_task( + client_id, + 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, + ); + let fetch_task_handle = tokio::spawn(fetch_task_future); + self.fetch_task_handles.insert(queue_id, fetch_task_handle); + Ok(()) + } + + pub fn unsubscribe( + &mut self, + index_uid: &str, + source_id: &str, + shard_id: u64, + ) -> IngestV2Result<()> { + let queue_id = queue_id(index_uid, source_id, shard_id); + + if let Some(fetch_stream_handle) = self.fetch_task_handles.remove(&queue_id) { + fetch_stream_handle.abort(); + } + Ok(()) + } + + pub async fn next(&mut self) -> Option> { + self.fetch_response_rx.recv().await + } +} + +/// Chooses the ingester to stream records from, preferring "local" ingesters. +fn select_preferred_and_failover_ingesters( + self_node_id: &NodeId, + leader_id: NodeId, + follower_id_opt: Option, +) -> (NodeId, Option) { + // The replication factor is 1 and there is no follower. + let Some(follower_id) = follower_id_opt else { + return (leader_id, None); + }; + if &leader_id == self_node_id { + (leader_id, Some(follower_id)) + } else if &follower_id == self_node_id { + (follower_id, Some(leader_id)) + } else if rand::random::() { + (leader_id, Some(follower_id)) + } else { + (follower_id, Some(leader_id)) + } +} + +/// Streams records from the preferred ingester and fails over to the other ingester if an error +/// occurs. +#[allow(clippy::too_many_arguments)] +async fn fetch_task( + client_id: String, + 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, + + 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; + } + } + Err(ingest_error) => { + if let Some(failover_ingester_id) = failover_ingester_id.take() { + 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}`." + ); + let mut ingester = ingester_pool + .get(&preferred_ingester_id) + .await + .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; + } + 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; + } + } + } +} + +#[cfg(test)] +mod tests { + use std::time::Duration; + + use bytes::Bytes; + use mrecordlog::MultiRecordLog; + use quickwit_proto::types::queue_id; + use tokio::time::timeout; + + use super::*; + + #[tokio::test] + async fn test_fetch_task() { + let tempdir = tempfile::tempdir().unwrap(); + let mrecordlog = Arc::new(RwLock::new( + MultiRecordLog::open(tempdir.path()).await.unwrap(), + )); + let client_id = "test-client".to_string(); + let index_uid = "test-index:0".to_string(); + let source_id = "test-source".to_string(); + let open_fetch_stream_request = OpenFetchStreamRequest { + client_id: client_id.clone(), + index_uid: index_uid.clone(), + source_id: source_id.clone(), + shard_id: 0, + from_position_exclusive: None, + to_position_inclusive: None, + }; + let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); + let mut fetch_stream = FetchTask::spawn( + open_fetch_stream_request, + mrecordlog.clone(), + shard_status_rx, + 1024, + ); + let mut mrecordlog_guard = mrecordlog.write().await; + + let queue_id = queue_id(&index_uid, &source_id, 0); + mrecordlog_guard.create_queue(&queue_id).await.unwrap(); + mrecordlog_guard + .append_record(&queue_id, None, Bytes::from_static(b"test-doc-000")) + .await + .unwrap(); + let shard_status = ShardStatus { + replication_position_inclusive: 0.into(), + ..Default::default() + }; + shard_status_tx.send(shard_status).unwrap(); + drop(mrecordlog_guard); + + let fetch_response = timeout(Duration::from_millis(100), 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, 0); + assert_eq!(fetch_response.from_position_inclusive, 0); + assert_eq!(fetch_response.doc_batch.as_ref().unwrap().doc_lengths, [12]); + assert_eq!( + fetch_response.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-000" + ); + + let mut mrecordlog_guard = mrecordlog.write().await; + + mrecordlog_guard + .append_record(&queue_id, None, Bytes::from_static(b"test-doc-001")) + .await + .unwrap(); + drop(mrecordlog_guard); + + timeout(Duration::from_millis(100), fetch_stream.next()) + .await + .unwrap_err(); + + let shard_status = ShardStatus { + replication_position_inclusive: 1.into(), + ..Default::default() + }; + shard_status_tx.send(shard_status).unwrap(); + + let fetch_response = timeout(Duration::from_millis(100), fetch_stream.next()) + .await + .unwrap() + .unwrap() + .unwrap(); + assert_eq!(fetch_response.from_position_inclusive, 1); + assert_eq!(fetch_response.doc_batch.as_ref().unwrap().doc_lengths, [12]); + assert_eq!( + fetch_response.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-001" + ); + } + + #[tokio::test] + async fn test_fetch_task_to_position() { + let tempdir = tempfile::tempdir().unwrap(); + let mrecordlog = Arc::new(RwLock::new( + MultiRecordLog::open(tempdir.path()).await.unwrap(), + )); + let client_id = "test-client".to_string(); + let index_uid = "test-index:0".to_string(); + let source_id = "test-source".to_string(); + let open_fetch_stream_request = OpenFetchStreamRequest { + client_id: client_id.clone(), + index_uid: index_uid.clone(), + source_id: source_id.clone(), + shard_id: 0, + from_position_exclusive: None, + to_position_inclusive: Some(0), + }; + let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); + let mut fetch_stream = FetchTask::spawn( + open_fetch_stream_request, + mrecordlog.clone(), + shard_status_rx, + 1024, + ); + let mut mrecordlog_guard = mrecordlog.write().await; + + let queue_id = queue_id(&index_uid, &source_id, 0); + mrecordlog_guard.create_queue(&queue_id).await.unwrap(); + mrecordlog_guard + .append_record(&queue_id, None, Bytes::from_static(b"test-doc-000")) + .await + .unwrap(); + + let shard_status = ShardStatus { + replication_position_inclusive: 0.into(), + ..Default::default() + }; + shard_status_tx.send(shard_status).unwrap(); + drop(mrecordlog_guard); + + let fetch_response = timeout(Duration::from_millis(100), 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, 0); + assert_eq!(fetch_response.from_position_inclusive, 0); + assert_eq!(fetch_response.doc_batch.as_ref().unwrap().doc_lengths, [12]); + assert_eq!( + fetch_response.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-000" + ); + } + + #[tokio::test] + async fn test_fetch_task_batch_num_bytes() { + let tempdir = tempfile::tempdir().unwrap(); + let mrecordlog = Arc::new(RwLock::new( + MultiRecordLog::open(tempdir.path()).await.unwrap(), + )); + let client_id = "test-client".to_string(); + let index_uid = "test-index:0".to_string(); + let source_id = "test-source".to_string(); + let open_fetch_stream_request = OpenFetchStreamRequest { + client_id: client_id.clone(), + index_uid: index_uid.clone(), + source_id: source_id.clone(), + shard_id: 0, + from_position_exclusive: None, + to_position_inclusive: Some(2), + }; + let (shard_status_tx, shard_status_rx) = watch::channel(ShardStatus::default()); + let mut fetch_stream = FetchTask::spawn( + open_fetch_stream_request, + mrecordlog.clone(), + shard_status_rx, + 30, + ); + let mut mrecordlog_guard = mrecordlog.write().await; + + let queue_id = queue_id(&index_uid, &source_id, 0); + mrecordlog_guard.create_queue(&queue_id).await.unwrap(); + mrecordlog_guard + .append_record(&queue_id, None, Bytes::from_static(b"test-doc-000")) + .await + .unwrap(); + mrecordlog_guard + .append_record(&queue_id, None, Bytes::from_static(b"test-doc-001")) + .await + .unwrap(); + mrecordlog_guard + .append_record(&queue_id, None, Bytes::from_static(b"test-doc-002")) + .await + .unwrap(); + + let shard_status = ShardStatus { + replication_position_inclusive: 2.into(), + ..Default::default() + }; + shard_status_tx.send(shard_status).unwrap(); + drop(mrecordlog_guard); + + let fetch_response = timeout(Duration::from_millis(100), 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, 0); + assert_eq!(fetch_response.from_position_inclusive, 0); + assert_eq!( + fetch_response.doc_batch.as_ref().unwrap().doc_lengths, + [12, 12] + ); + assert_eq!( + fetch_response.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-000test-doc-001" + ); + let fetch_response = timeout(Duration::from_millis(100), 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, 0); + assert_eq!(fetch_response.from_position_inclusive, 2); + assert_eq!(fetch_response.doc_batch.as_ref().unwrap().doc_lengths, [12]); + assert_eq!( + fetch_response.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-002" + ); + } + + #[tokio::test] + async fn test_fetch_task_failover() { + // TODO: Backport from original branch. + } + + #[tokio::test] + async fn test_multi_fetch_stream() { + let 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); + // TODO: Backport from original branch. + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs new file mode 100644 index 00000000000..317d0fd9d7d --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -0,0 +1,1273 @@ +// 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::collections::hash_map::Entry; +use std::collections::HashMap; +use std::iter::once; +use std::path::Path; +use std::sync::Arc; +use std::time::Duration; +use std::{cmp, fmt}; + +use async_trait::async_trait; +use bytes::{Bytes, BytesMut}; +use futures::stream::FuturesUnordered; +use futures::StreamExt; +use mrecordlog::MultiRecordLog; +use quickwit_common::tower::Pool; +use quickwit_common::ServiceStream; +use quickwit_proto::ingest::ingester::{ + AckReplicationMessage, FetchResponseV2, IngesterService, IngesterServiceClient, + IngesterServiceStream, OpenFetchStreamRequest, OpenReplicationStreamRequest, + OpenReplicationStreamResponse, PersistRequest, PersistResponse, PersistSuccess, PingRequest, + PingResponse, ReplicateRequest, ReplicateSubrequest, SynReplicationMessage, TruncateRequest, + TruncateResponse, TruncateSubrequest, +}; +use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result, ShardState}; +use quickwit_proto::types::{NodeId, QueueId}; +use tokio::sync::{watch, RwLock}; + +use super::fetch::FetchTask; +use super::replication::{ + ReplicationClient, ReplicationClientTask, ReplicationTask, ReplicationTaskHandle, +}; +use super::IngesterPool; +use crate::metrics::INGEST_METRICS; +use crate::DocCommand; + +#[derive(Clone)] +pub struct Ingester { + self_node_id: NodeId, + ingester_pool: IngesterPool, + mrecordlog: Arc>, + state: Arc>, + replication_factor: usize, +} + +impl fmt::Debug for Ingester { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Ingester") + .field("replication_factor", &self.replication_factor) + .finish() + } +} + +pub(super) struct IngesterState { + pub primary_shards: HashMap, + pub replica_shards: HashMap, + pub replication_clients: HashMap, + 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, + ingester_pool: Pool, + wal_dir_path: &Path, + replication_factor: usize, + ) -> IngestV2Result { + let mrecordlog = MultiRecordLog::open_with_prefs( + wal_dir_path, + mrecordlog::SyncPolicy::OnDelay(Duration::from_secs(5)), + ) + .await + .map_err(|error| IngestV2Error::Internal(error.to_string()))?; + let mrecordlog = Arc::new(RwLock::new(mrecordlog)); + + let inner = IngesterState { + primary_shards: HashMap::new(), + replica_shards: HashMap::new(), + replication_clients: HashMap::new(), + replication_tasks: HashMap::new(), + }; + let ingester = Self { + self_node_id, + ingester_pool, + mrecordlog, + state: Arc::new(RwLock::new(inner)), + replication_factor, + }; + Ok(ingester) + } + + async fn init_primary_shard<'a>( + &self, + state: &'a mut IngesterState, + queue_id: &QueueId, + leader_id: &NodeId, + follower_id_opt: Option<&NodeId>, + ) -> IngestV2Result<&'a PrimaryShard> { + let mut mrecordlog_guard = self.mrecordlog.write().await; + + if !mrecordlog_guard.queue_exists(queue_id) { + mrecordlog_guard.create_queue(queue_id).await.expect("TODO"); // IO error, what to do? + } else { + // TODO: Recover last position from mrecordlog and take it from there. + } + drop(mrecordlog_guard); + + if let Some(follower_id) = follower_id_opt { + self.init_replication_client(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 entry = state.primary_shards.entry(queue_id.clone()); + Ok(entry.or_insert(primary_shard)) + } + + async fn init_replication_client( + &self, + state: &mut IngesterState, + leader_id: &NodeId, + follower_id: &NodeId, + ) -> IngestV2Result<()> { + let Entry::Vacant(entry) = state.replication_clients.entry(follower_id.clone()) else { + // The replication client is already initialized. Nothing to do! + return Ok(()); + }; + let open_request = OpenReplicationStreamRequest { + leader_id: leader_id.clone().into(), + follower_id: follower_id.clone().into(), + }; + let open_message = SynReplicationMessage::new_open_request(open_request); + let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(5); + syn_replication_stream_tx + .try_send(open_message) + .expect("The channel should be open and have capacity."); + + let mut ingester = self.ingester_pool.get(follower_id).await.ok_or( + IngestV2Error::IngesterUnavailable { + ingester_id: follower_id.clone(), + }, + )?; + let mut ack_replication_stream = ingester + .open_replication_stream(syn_replication_stream) + .await?; + ack_replication_stream + .next() + .await + .expect("TODO") + .expect("") + .into_open_response() + .expect("The first message should be an open response."); + + let replication_client = + ReplicationClientTask::spawn(syn_replication_stream_tx, ack_replication_stream); + entry.insert(replication_client); + Ok(()) + } +} + +#[async_trait] +impl IngesterService for Ingester { + async fn persist( + &mut self, + persist_request: PersistRequest, + ) -> IngestV2Result { + if persist_request.leader_id != self.self_node_id { + return Err(IngestV2Error::Internal(format!( + "Routing error: request was sent to ingester node `{}` instead of `{}`.", + self.self_node_id, persist_request.leader_id, + ))); + } + let mut state_guard = self.state.write().await; + + let mut persist_successes = Vec::with_capacity(persist_request.subrequests.len()); + // let mut persist_failures = Vec::new(); + let mut replicate_subrequests: HashMap> = HashMap::new(); + + let commit_type = persist_request.commit_type(); + let force_commit = commit_type == CommitTypeV2::Force; + let leader_id: NodeId = persist_request.leader_id.into(); + + 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? + }; + let from_position_inclusive = primary_shard.primary_position_inclusive; + + 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 mut mrecordlog_guard = self.mrecordlog.write().await; + + let primary_position_inclusive = if force_commit { + let docs = doc_batch.docs().chain(once(commit_doc())); + mrecordlog_guard + .append_records(&queue_id, None, docs) + .await + .expect("TODO") // TODO: Io error, close shard? + } else { + let docs = doc_batch.docs(); + mrecordlog_guard + .append_records(&queue_id, None, docs) + .await + .expect("TODO") // TODO: Io error, close shard? + }; + let batch_num_bytes = doc_batch.num_bytes() as u64; + let batch_num_docs = doc_batch.num_docs() as u64; + + INGEST_METRICS.ingested_num_bytes.inc_by(batch_num_bytes); + INGEST_METRICS.ingested_num_docs.inc_by(batch_num_docs); + + state_guard + .primary_shards + .get_mut(&queue_id) + .expect("Primary shard should exist.") + .set_primary_position_inclusive(primary_position_inclusive); + + if let Some(follower_id) = follower_id { + 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(), + doc_batch: Some(doc_batch), + }; + replicate_subrequests + .entry(follower_id) + .or_default() + .push(replicate_subrequest); + } else { + let persist_success = PersistSuccess { + index_uid: subrequest.index_uid, + source_id: subrequest.source_id, + shard_id: subrequest.shard_id, + replication_position_inclusive: primary_position_inclusive, + }; + persist_successes.push(persist_success); + } + } + if replicate_subrequests.is_empty() { + let leader_id = self.self_node_id.to_string(); + let persist_response = PersistResponse { + leader_id, + successes: persist_successes, + failures: Vec::new(), // TODO + }; + return Ok(persist_response); + } + let mut replicate_futures = FuturesUnordered::new(); + + for (follower_id, subrequests) in replicate_subrequests { + let replicate_request = ReplicateRequest { + leader_id: self.self_node_id.clone().into(), + follower_id: follower_id.clone().into(), + subrequests, + commit_type: persist_request.commit_type, + }; + let replication_client = state_guard + .replication_clients + .get(&follower_id) + .expect("The replication client should be initialized.") + .clone(); + replicate_futures + .push(async move { replication_client.replicate(replicate_request).await }); + } + // Drop the write lock AFTER pushing the replicate request into the replication client + // channel to ensure that sequential writes in mrecordlog turn into sequential replicate + // requests in the same order. + drop(state_guard); + + while let Some(replicate_result) = replicate_futures.next().await { + let replicate_response = replicate_result?; + + for replicate_success in replicate_response.successes { + let persist_success = PersistSuccess { + 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, + }; + persist_successes.push(persist_success); + } + } + let mut 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 leader_id = self.self_node_id.to_string(); + let persist_response = PersistResponse { + leader_id, + successes: persist_successes, + failures: Vec::new(), // TODO + }; + Ok(persist_response) + } + + /// Opens a replication stream, which is a bi-directional gRPC stream. The client-side stream + async fn open_replication_stream( + &mut self, + mut syn_replication_stream: quickwit_common::ServiceStream, + ) -> IngestV2Result> { + let open_replication_stream_request = syn_replication_stream + .next() + .await + .ok_or_else(|| IngestV2Error::Internal("Syn replication stream aborted.".to_string()))? + .into_open_request() + .expect("The first message should be an open replication stream request."); + + if open_replication_stream_request.follower_id != self.self_node_id { + return Err(IngestV2Error::Internal("Routing error".to_string())); + } + let leader_id: NodeId = open_replication_stream_request.leader_id.into(); + let follower_id: NodeId = open_replication_stream_request.follower_id.into(); + + let mut state_guard = self.state.write().await; + + let Entry::Vacant(entry) = state_guard.replication_tasks.entry(leader_id.clone()) else { + return Err(IngestV2Error::Internal(format!( + "A replication stream betwen {leader_id} and {follower_id} is already opened." + ))); + }; + let (ack_replication_stream_tx, ack_replication_stream) = ServiceStream::new_bounded(5); + let open_response = OpenReplicationStreamResponse {}; + let ack_replication_message = AckReplicationMessage::new_open_response(open_response); + ack_replication_stream_tx + .send(Ok(ack_replication_message)) + .await + .expect("Channel should be open and have enough capacity."); + + let replication_task_handle = ReplicationTask::spawn( + leader_id, + follower_id, + self.mrecordlog.clone(), + self.state.clone(), + syn_replication_stream, + ack_replication_stream_tx, + ); + entry.insert(replication_task_handle); + Ok(ack_replication_stream) + } + + async fn open_fetch_stream( + &mut self, + open_fetch_stream_request: OpenFetchStreamRequest, + ) -> IngestV2Result>> { + let mrecordlog = self.mrecordlog.clone(); + let queue_id = open_fetch_stream_request.queue_id(); + let shard_status_rx = self + .state + .read() + .await + .find_shard_status_rx(&queue_id) + .ok_or_else(|| IngestV2Error::Internal("Shard not found.".to_string()))?; + let service_stream = FetchTask::spawn( + open_fetch_stream_request, + mrecordlog, + shard_status_rx, + FetchTask::DEFAULT_BATCH_NUM_BYTES, + ); + Ok(service_stream) + } + + async fn ping(&mut self, ping_request: PingRequest) -> IngestV2Result { + if ping_request.leader_id != self.self_node_id { + let ping_response = PingResponse {}; + return Ok(ping_response); + }; + let Some(follower_id) = &ping_request.follower_id else { + let ping_response = PingResponse {}; + return Ok(ping_response); + }; + let follower_id: NodeId = follower_id.clone().into(); + let mut ingester = self.ingester_pool.get(&follower_id).await.ok_or_else(|| { + IngestV2Error::Internal(format!("No ingester found for follower `{}`.", follower_id)) + })?; + ingester.ping(ping_request).await?; + let ping_response = PingResponse {}; + Ok(ping_response) + } + + async fn truncate( + &mut self, + truncate_request: TruncateRequest, + ) -> IngestV2Result { + if truncate_request.leader_id != self.self_node_id { + return Err(IngestV2Error::Internal(format!( + "Routing error: expected ingester `{}`, got `{}`.", + truncate_request.leader_id, self.self_node_id + ))); + } + let state_guard = self.state.write().await; + + let mut truncate_subrequests: HashMap> = HashMap::new(); + + for subrequest in truncate_request.subrequests { + let queue_id = subrequest.queue_id(); + + if state_guard.primary_shards.contains_key(&queue_id) { + let mut mrecordlog_guard = self.mrecordlog.write().await; + mrecordlog_guard + .truncate(&queue_id, subrequest.to_position_inclusive) + .await + .map_err(|error| { + IngestV2Error::Internal(format!("Failed to truncate: {error:?}")) + })?; + } + if let Some(replica_shard) = state_guard.replica_shards.get(&queue_id) { + truncate_subrequests + .entry(replica_shard.leader_id.clone()) + .or_default() + .push(subrequest); + } + } + let mut truncate_futures = FuturesUnordered::new(); + + for (follower_id, subrequests) in truncate_subrequests { + let leader_id = self.self_node_id.clone().into(); + let truncate_request = TruncateRequest { + leader_id, + subrequests, + }; + let replication_client = state_guard + .replication_clients + .get(&follower_id) + .expect("The replication client should be initialized.") + .clone(); + truncate_futures + .push(async move { replication_client.truncate(truncate_request).await }); + } + // Drop the write lock AFTER pushing the replicate request into the replication client + // channel to ensure that sequential writes in mrecordlog turn into sequential replicate + // requests in the same order. + drop(state_guard); + + while let Some(truncate_result) = truncate_futures.next().await { + // TODO: Handle errors. + truncate_result?; + } + let truncate_response = TruncateResponse {}; + Ok(truncate_response) + } +} + +#[derive(Debug, Clone, Eq, PartialEq)] +pub(super) struct ShardStatus { + /// Current state of the shard. + 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. + pub replication_position_inclusive: Position, +} + +impl Default for ShardStatus { + fn default() -> Self { + Self { + shard_state: ShardState::Open, + publish_position_inclusive: Position::default(), + replication_position_inclusive: Position::default(), + } + } +} + +#[derive(Debug, Clone, Copy, Default, Eq, PartialEq, Ord, PartialOrd, Hash)] +pub(super) enum Position { + #[default] + Beginning, + Offset(u64), +} + +impl Position { + pub fn offset(&self) -> Option { + match self { + Position::Beginning => None, + Position::Offset(offset) => Some(*offset), + } + } +} + +impl PartialEq for Position { + fn eq(&self, other: &u64) -> bool { + match self { + Position::Beginning => false, + Position::Offset(offset) => offset == other, + } + } +} + +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), + } + } +} + +impl From for Position { + fn from(offset: u64) -> Self { + Position::Offset(offset) + } +} + +impl From> for Position { + fn from(offset_opt: Option) -> Self { + match offset_opt { + Some(offset) => Position::Offset(offset), + None => Position::Beginning, + } + } +} + +/// 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. + _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. + _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, +} + +impl PrimaryShard { + // TODO: Set publish position on truncate. + // fn set_publish_position_inclusive(&mut self, publish_position_inclusive: impl Into) + // { self.publish_position_inclusive = publish_position_inclusive.into(); + + // if self.shard_state.is_closed() { + // self.shard_status_tx.send_modify(|shard_status| { + // shard_status.shard_state = self.shard_state; + // shard_status.publish_position_inclusive = self.publish_position_inclusive + // }); + // } + // } + + 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 + }) + } + } + + 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(super) _shard_state: ShardState, + pub(super) _publish_position_inclusive: Position, + pub replica_position_inclusive: Position, + pub shard_status_tx: watch::Sender, + pub shard_status_rx: watch::Receiver, +} + +impl ReplicaShard { + // TODO: Set publish position on truncate. + // fn set_publish_position_inclusive(&mut self, publish_position_inclusive: impl Into) + // { self.publish_position_inclusive = publish_position_inclusive.into(); + + // if self.shard_state.is_closed() { + // self.shard_status_tx.send_modify(|shard_status| { + // shard_status.shard_state = self.shard_state; + // 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 + }); + } +} + +// TODO +pub(super) fn commit_doc() -> Bytes { + let mut buffer = BytesMut::with_capacity(1); + let command = DocCommand::::Commit; + command.write(&mut buffer); + Bytes::from(buffer) +} + +#[cfg(test)] +mod tests { + use std::net::SocketAddr; + use std::time::Duration; + + use quickwit_proto::ingest::ingester::{ + IngesterServiceGrpcServer, IngesterServiceGrpcServerAdapter, PersistSubrequest, + }; + use quickwit_proto::ingest::DocBatchV2; + use quickwit_proto::types::queue_id; + use tonic::transport::{Endpoint, Server}; + use tower::timeout::Timeout; + + use super::*; + use crate::ingest_v2::test_utils::{ + MultiRecordLogTestExt, PrimaryShardTestExt, ReplicaShardTestExt, + }; + + const NONE_REPLICA_POSITION: Option = None; + + #[tokio::test] + async fn test_ingester_persist() { + let tempdir = tempfile::tempdir().unwrap(); + let node_id: NodeId = "test-ingester-0".into(); + let ingester_pool = IngesterPool::default(); + let wal_dir_path = tempdir.path(); + let replication_factor = 1; + let mut ingester = Ingester::try_new( + node_id.clone(), + ingester_pool, + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + let persist_request = PersistRequest { + leader_id: node_id.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: None, + doc_batch: None, + }, + PersistSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + follower_id: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-010"), + doc_lengths: vec![12], + }), + }, + PersistSubrequest { + index_uid: "test-index:1".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + follower_id: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-100test-doc-101"), + doc_lengths: vec![12, 12], + }), + }, + ], + }; + ingester.persist(persist_request).await.unwrap(); + + let state_guard = ingester.state.read().await; + let mrecordlog_guard = ingester.mrecordlog.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); + + mrecordlog_guard.assert_records_eq(&queue_id_00, .., &[]); + + 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(0, NONE_REPLICA_POSITION); + primary_shard_01.assert_is_open(0); + + mrecordlog_guard.assert_records_eq(&queue_id_01, .., &[(0, "test-doc-010")]); + + 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(1, NONE_REPLICA_POSITION); + primary_shard_10.assert_is_open(1); + + mrecordlog_guard.assert_records_eq( + &queue_id_10, + .., + &[(0, "test-doc-100"), (1, "test-doc-101")], + ); + } + + #[tokio::test] + async fn test_ingester_open_replication_stream() { + let tempdir = tempfile::tempdir().unwrap(); + let node_id: NodeId = "test-follower".into(); + let ingester_pool = IngesterPool::default(); + let wal_dir_path = tempdir.path(); + let replication_factor = 1; + let mut ingester = Ingester::try_new( + node_id.clone(), + ingester_pool, + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(5); + let open_stream_request = OpenReplicationStreamRequest { + leader_id: "test-leader".to_string(), + follower_id: "test-follower".to_string(), + }; + let syn_replication_message = SynReplicationMessage::new_open_request(open_stream_request); + syn_replication_stream_tx + .send(syn_replication_message) + .await + .unwrap(); + let mut ack_replication_stream = ingester + .open_replication_stream(syn_replication_stream) + .await + .unwrap(); + ack_replication_stream + .next() + .await + .unwrap() + .unwrap() + .into_open_response() + .unwrap(); + + let state_guard = ingester.state.read().await; + assert!(state_guard.replication_tasks.contains_key("test-leader")); + } + + #[tokio::test] + async fn test_ingester_persist_replicate() { + let tempdir = tempfile::tempdir().unwrap(); + let leader_id: NodeId = "test-leader".into(); + let ingester_pool = IngesterPool::default(); + let wal_dir_path = tempdir.path(); + let replication_factor = 2; + let mut leader = Ingester::try_new( + leader_id.clone(), + ingester_pool.clone(), + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + let tempdir = tempfile::tempdir().unwrap(); + let follower_id: NodeId = "test-follower".into(); + let wal_dir_path = tempdir.path(); + let replication_factor = 2; + let follower = Ingester::try_new( + follower_id.clone(), + ingester_pool.clone(), + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + ingester_pool + .insert( + follower_id.clone(), + IngesterServiceClient::new(follower.clone()), + ) + .await; + + let persist_request = PersistRequest { + 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(), + shard_id: 1, + follower_id: Some(follower_id.to_string()), + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-010"), + doc_lengths: vec![12], + }), + }, + PersistSubrequest { + index_uid: "test-index:1".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + follower_id: Some(follower_id.to_string()), + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-100test-doc-101"), + doc_lengths: vec![12, 12], + }), + }, + ], + }; + 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.failures.len(), 0); + + let leader_state_guard = leader.state.read().await; + let leader_mrecordlog_guard = leader.mrecordlog.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_mrecordlog_guard.assert_records_eq(&queue_id_00, .., &[]); + + 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); + + leader_mrecordlog_guard.assert_records_eq(&queue_id_01, .., &[(0, "test-doc-010")]); + + 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); + + leader_mrecordlog_guard.assert_records_eq( + &queue_id_10, + .., + &[(0, "test-doc-100"), (1, "test-doc-101")], + ); + } + + #[tokio::test] + async fn test_ingester_persist_replicate_grpc() { + let tempdir = tempfile::tempdir().unwrap(); + let leader_id: NodeId = "test-leader".into(); + let ingester_pool = IngesterPool::default(); + let wal_dir_path = tempdir.path(); + let replication_factor = 2; + let mut leader = Ingester::try_new( + leader_id.clone(), + ingester_pool.clone(), + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + let leader_grpc_server_adapter = IngesterServiceGrpcServerAdapter::new(leader.clone()); + let leader_grpc_server = IngesterServiceGrpcServer::new(leader_grpc_server_adapter); + let leader_socket_addr: SocketAddr = "127.0.0.1:6666".parse().unwrap(); + + tokio::spawn({ + async move { + Server::builder() + .add_service(leader_grpc_server) + .serve(leader_socket_addr) + .await + .unwrap(); + } + }); + + let tempdir = tempfile::tempdir().unwrap(); + let follower_id: NodeId = "test-follower".into(); + let wal_dir_path = tempdir.path(); + let replication_factor = 2; + let follower = Ingester::try_new( + follower_id.clone(), + ingester_pool.clone(), + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + let follower_grpc_server_adapter = IngesterServiceGrpcServerAdapter::new(follower.clone()); + let follower_grpc_server = IngesterServiceGrpcServer::new(follower_grpc_server_adapter); + let follower_socket_addr: SocketAddr = "127.0.0.1:7777".parse().unwrap(); + + tokio::spawn({ + async move { + Server::builder() + .add_service(follower_grpc_server) + .serve(follower_socket_addr) + .await + .unwrap(); + } + }); + let follower_channel = Timeout::new( + Endpoint::from_static("http://127.0.0.1:7777").connect_lazy(), + Duration::from_secs(1), + ); + let follower_grpc_client = IngesterServiceClient::from_channel(follower_channel); + + ingester_pool + .insert(follower_id.clone(), follower_grpc_client) + .await; + + let persist_request = PersistRequest { + 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(), + shard_id: 1, + follower_id: Some(follower_id.to_string()), + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-010"), + doc_lengths: vec![12], + }), + }, + PersistSubrequest { + index_uid: "test-index:1".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + follower_id: Some(follower_id.to_string()), + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-100test-doc-101"), + doc_lengths: vec![12, 12], + }), + }, + ], + }; + 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.failures.len(), 0); + + let queue_id_00 = queue_id("test-index:0", "test-source", 0); + + let leader_state_guard = leader.state.read().await; + let leader_mrecordlog_guard = leader.mrecordlog.read().await; + leader_mrecordlog_guard.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; + let follower_mrecordlog_guard = follower.mrecordlog.read().await; + assert!(!follower_mrecordlog_guard.queue_exists(&queue_id_00)); + + assert!(!follower_state_guard + .replica_shards + .contains_key(&queue_id_00)); + + let queue_id_01 = queue_id("test-index:0", "test-source", 1); + + let leader_state_guard = leader.state.read().await; + leader_mrecordlog_guard.assert_records_eq(&queue_id_01, .., &[(0, "test-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); + + follower_mrecordlog_guard.assert_records_eq(&queue_id_01, .., &[(0, "test-doc-010")]); + + 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 queue_id_10 = queue_id("test-index:1", "test-source", 0); + + leader_mrecordlog_guard.assert_records_eq( + &queue_id_10, + .., + &[(0, "test-doc-100"), (1, "test-doc-101")], + ); + + 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); + + follower_mrecordlog_guard.assert_records_eq( + &queue_id_10, + .., + &[(0, "test-doc-100"), (1, "test-doc-101")], + ); + + 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 node_id: NodeId = "test-ingester-0".into(); + let ingester_pool = IngesterPool::default(); + let wal_dir_path = tempdir.path(); + let replication_factor = 1; + let mut ingester = Ingester::try_new( + node_id.clone(), + ingester_pool, + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + let persist_request = PersistRequest { + leader_id: node_id.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: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-000"), + doc_lengths: vec![12], + }), + }, + PersistSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + follower_id: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-010"), + doc_lengths: vec![12], + }), + }, + ], + }; + 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, + from_position_exclusive: None, + to_position_inclusive: None, + }; + let mut fetch_stream = ingester + .open_fetch_stream(open_fetch_stream_request) + .await + .unwrap(); + let fetch_response = fetch_stream.next().await.unwrap().unwrap(); + let doc_batch = fetch_response.doc_batch.unwrap(); + assert_eq!(doc_batch.doc_buffer, Bytes::from_static(b"test-doc-000")); + assert_eq!(doc_batch.doc_lengths, [12]); + assert_eq!(fetch_response.from_position_inclusive, 0); + + let persist_request = PersistRequest { + leader_id: node_id.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: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-001test-doc-002"), + doc_lengths: vec![12, 12], + }), + }], + }; + ingester.persist(persist_request).await.unwrap(); + + let fetch_response = fetch_stream.next().await.unwrap().unwrap(); + let doc_batch = fetch_response.doc_batch.unwrap(); + assert_eq!( + doc_batch.doc_buffer, + Bytes::from_static(b"test-doc-001test-doc-002") + ); + assert_eq!(doc_batch.doc_lengths, [12, 12]); + assert_eq!(fetch_response.from_position_inclusive, 1); + } + + #[tokio::test] + async fn test_ingester_truncate() { + let tempdir = tempfile::tempdir().unwrap(); + let node_id: NodeId = "test-ingester-0".into(); + let ingester_pool = IngesterPool::default(); + let wal_dir_path = tempdir.path(); + let replication_factor = 1; + let mut ingester = Ingester::try_new( + node_id.clone(), + ingester_pool, + wal_dir_path, + replication_factor, + ) + .await + .unwrap(); + + let queue_id_00 = queue_id("test-index:0", "test-source", 0); + let queue_id_01 = queue_id("test-index:0", "test-source", 1); + + let mut ingester_state = ingester.state.write().await; + ingester + .init_primary_shard(&mut ingester_state, &queue_id_00, &node_id, None) + .await + .unwrap(); + ingester + .init_primary_shard(&mut ingester_state, &queue_id_01, &node_id, None) + .await + .unwrap(); + + drop(ingester_state); + + let mut mrecordlog_guard = ingester.mrecordlog.write().await; + + let records = [ + Bytes::from_static(b"test-doc-000"), + Bytes::from_static(b"test-doc-001"), + ] + .into_iter(); + mrecordlog_guard + .append_records(&queue_id_00, None, records) + .await + .unwrap(); + + let records = [ + Bytes::from_static(b"test-doc-010"), + Bytes::from_static(b"test-doc-011"), + ] + .into_iter(); + mrecordlog_guard + .append_records(&queue_id_00, None, records) + .await + .unwrap(); + + drop(mrecordlog_guard); + + let truncate_request = TruncateRequest { + leader_id: node_id.to_string(), + subrequests: vec![ + TruncateSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + to_position_inclusive: 0, + }, + TruncateSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + to_position_inclusive: 1, + }, + TruncateSubrequest { + index_uid: "test-index:1337".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + to_position_inclusive: 1337, + }, + ], + }; + ingester.truncate(truncate_request).await.unwrap(); + + let mrecordlog_guard = ingester.mrecordlog.read().await; + let (position, record) = mrecordlog_guard + .range(&queue_id_00, 0..) + .unwrap() + .next() + .unwrap(); + assert_eq!(position, 1); + assert_eq!(&*record, b"test-doc-001"); + + let record_opt = mrecordlog_guard.range(&queue_id_01, 0..).unwrap().next(); + assert!(record_opt.is_none()); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/mod.rs b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs new file mode 100644 index 00000000000..a5b57bc79e1 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/mod.rs @@ -0,0 +1,80 @@ +// 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 . + +mod fetch; +mod ingester; +mod replication; +mod router; +mod shard_table; +#[cfg(test)] +mod test_utils; + +use bytes::{BufMut, BytesMut}; +use quickwit_common::tower::Pool; +use quickwit_proto::ingest::ingester::IngesterServiceClient; +use quickwit_proto::ingest::DocBatchV2; +use quickwit_proto::types::NodeId; + +pub use self::fetch::MultiFetchStream; +pub use self::ingester::Ingester; +pub use self::router::IngestRouter; + +pub type IngesterPool = Pool; + +/// Identifies an ingester client, typically a source, for logging and debugging purposes. +pub type ClientId = String; + +#[derive(Default)] +pub(crate) struct DocBatchBuilderV2 { + doc_buffer: BytesMut, + doc_lengths: Vec, +} + +impl DocBatchBuilderV2 { + pub fn with_capacity(capacity: usize) -> Self { + Self { + doc_buffer: BytesMut::with_capacity(capacity), + doc_lengths: Vec::new(), + } + } + + pub fn add_doc(&mut self, doc: &[u8]) { + self.doc_lengths.push(doc.len() as u32); + self.doc_buffer.put(doc); + } + + pub fn build(self) -> DocBatchV2 { + DocBatchV2 { + doc_buffer: self.doc_buffer.freeze(), + doc_lengths: self.doc_lengths, + } + } + + pub fn capacity(&self) -> usize { + self.doc_buffer.capacity() + } + + pub fn is_empty(&self) -> bool { + self.doc_lengths.is_empty() + } + + fn num_bytes(&self) -> usize { + self.doc_buffer.len() + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/replication.rs b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs new file mode 100644 index 00000000000..8f315b16c69 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/replication.rs @@ -0,0 +1,687 @@ +// 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::iter::once; +use std::sync::Arc; + +use futures::StreamExt; +use mrecordlog::MultiRecordLog; +use quickwit_common::ServiceStream; +use quickwit_proto::ingest::ingester::{ + ack_replication_message, syn_replication_message, AckReplicationMessage, ReplicateRequest, + ReplicateResponse, ReplicateSuccess, SynReplicationMessage, TruncateRequest, TruncateResponse, +}; +use quickwit_proto::ingest::{CommitTypeV2, IngestV2Error, IngestV2Result, ShardState}; +use quickwit_proto::types::NodeId; +use tokio::sync::{mpsc, oneshot, watch, RwLock}; +use tokio::task::JoinHandle; + +use super::ingester::{commit_doc, IngesterState, Position, ReplicaShard, ShardStatus}; +use crate::metrics::INGEST_METRICS; + +/// A replication request is sent by the leader to its follower to update the state of a replica +/// shard. +#[derive(Debug)] +pub(super) enum ReplicationRequest { + Replicate(ReplicateRequest), + Truncate(TruncateRequest), +} + +#[derive(Debug)] +pub(super) enum ReplicationResponse { + Replicate(ReplicateResponse), + Truncate(TruncateResponse), +} + +impl ReplicationResponse { + pub fn into_replicate_response(self) -> Option { + match self { + ReplicationResponse::Replicate(replicate_response) => Some(replicate_response), + ReplicationResponse::Truncate(_) => None, + } + } + + pub fn into_truncate_response(self) -> Option { + match self { + ReplicationResponse::Replicate(_) => None, + ReplicationResponse::Truncate(truncate_response) => Some(truncate_response), + } + } +} + +type OneShotReplicationRequest = (ReplicationRequest, oneshot::Sender); + +/// Offers a request-response API on top of a gRPC bi-directional replication stream. There should +/// be one replication client per leader-follower pair. +#[derive(Clone)] +pub(super) struct ReplicationClient { + oneshot_replication_request_tx: mpsc::UnboundedSender, +} + +impl ReplicationClient { + /// Replicates a persist request from a leader to its follower and waits for its response. + pub async fn replicate( + &self, + replicate_request: ReplicateRequest, + ) -> IngestV2Result { + let replication_request = ReplicationRequest::Replicate(replicate_request); + let (replication_response_tx, replication_response_rx) = oneshot::channel(); + self.oneshot_replication_request_tx + .clone() + .send((replication_request, replication_response_tx)) + .expect("TODO"); + let replicate_response = replication_response_rx + .await + .expect("TODO") + .into_replicate_response() + .expect("TODO"); + Ok(replicate_response) + } + + /// Replicates a truncate request from a leader to its follower and waits for its response. + pub async fn truncate( + &self, + truncate_request: TruncateRequest, + ) -> IngestV2Result { + let replication_request = ReplicationRequest::Truncate(truncate_request); + let (replication_response_tx, replication_response_rx) = oneshot::channel(); + self.oneshot_replication_request_tx + .clone() + .send((replication_request, replication_response_tx)) + .expect("TODO"); + let truncate_response = replication_response_rx + .await + .expect("TODO") + .into_truncate_response() + .expect("TODO"); + Ok(truncate_response) + } +} + +/// Processes [`ReplicateRequest`] requests sent by a leader. It queues requests and pipes them into +/// its underlying replication stream, then waits for responses from the follower on the other end +/// of the stream. +pub(super) struct ReplicationClientTask { + syn_replication_stream_tx: mpsc::Sender, + ack_replication_stream: ServiceStream>, + oneshot_replication_request_rx: mpsc::UnboundedReceiver, +} + +impl ReplicationClientTask { + /// Spawns a [`ReplicationClientTask`]. + pub fn spawn( + syn_replication_stream_tx: mpsc::Sender, + ack_replication_stream: ServiceStream>, + ) -> ReplicationClient { + let (oneshot_replication_request_tx, oneshot_replication_request_rx) = + mpsc::unbounded_channel::(); // TODO: bound and handle backpressure on the other side. + + let mut replication_client_task = Self { + syn_replication_stream_tx, + ack_replication_stream, + oneshot_replication_request_rx, + }; + let future = async move { + replication_client_task.run().await; + }; + tokio::spawn(future); + + ReplicationClient { + oneshot_replication_request_tx, + } + } + + /// Executes the processing loop. + // TODO: There is a major flaw in this implementation: it processes requests sequentially, while + // it should be able to enqueue incoming replication requests while waiting for the next + // replication response from the follower. + async fn run(&mut self) { + while let Some((replication_request, replication_response_tx)) = + self.oneshot_replication_request_rx.recv().await + { + // TODO: Batch requests. + let syn_replication_message = match replication_request { + ReplicationRequest::Replicate(replication_request) => { + SynReplicationMessage::new_replicate_request(replication_request) + } + ReplicationRequest::Truncate(truncate_request) => { + SynReplicationMessage::new_truncate_request(truncate_request) + } + }; + self.syn_replication_stream_tx + .send(syn_replication_message) + .await + .expect("TODO"); + let ack_replication_message = self + .ack_replication_stream + .next() + .await + .expect("TODO") + .expect("TODO"); + let replication_response = + into_replication_response(ack_replication_message).expect(""); + replication_response_tx + .send(replication_response) + .expect("TODO"); + } + } +} + +pub(super) struct ReplicationTaskHandle { + _join_handle: JoinHandle>, +} + +/// Replication task executed per replication stream. +pub(super) struct ReplicationTask { + leader_id: NodeId, + follower_id: NodeId, + mrecordlog: Arc>, + state: Arc>, + syn_replication_stream: ServiceStream, + ack_replication_stream_tx: mpsc::Sender>, +} + +impl ReplicationTask { + pub fn spawn( + leader_id: NodeId, + follower_id: NodeId, + mrecordlog: Arc>, + state: Arc>, + syn_replication_stream: ServiceStream, + ack_replication_stream_tx: mpsc::Sender>, + ) -> ReplicationTaskHandle { + let mut replication_task = Self { + leader_id, + follower_id, + mrecordlog, + state, + syn_replication_stream, + ack_replication_stream_tx, + }; + let future = async move { replication_task.run().await }; + let _join_handle = tokio::spawn(future); + ReplicationTaskHandle { _join_handle } + } + + async fn replicate( + &mut self, + replicate_request: ReplicateRequest, + ) -> IngestV2Result { + if replicate_request.leader_id != self.leader_id { + return Err(IngestV2Error::Internal(format!( + "Invalid argument: expected leader ID `{}`, got `{}`.", + self.leader_id, replicate_request.leader_id + ))); + } + if replicate_request.follower_id != self.follower_id { + return Err(IngestV2Error::Internal(format!( + "Invalid argument: expected follower ID `{}`, got `{}`.", + self.follower_id, replicate_request.follower_id + ))); + } + let commit_type = replicate_request.commit_type(); + let force_commit = commit_type == CommitTypeV2::Force; + let mut replicate_successes = Vec::with_capacity(replicate_request.subrequests.len()); + + let mut state_guard = self.state.write().await; + + for subrequest in replicate_request.subrequests { + let queue_id = subrequest.queue_id(); + + let _replica_shard: &mut ReplicaShard = if subrequest.from_position_exclusive.is_none() + { + let mut mrecordlog_guard = self.mrecordlog.write().await; + // Initialize the replica shard and corresponding mrecordlog queue. + mrecordlog_guard + .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("The replica shard should be initialized.") + }; + 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 mut mrecordlog_guard = self.mrecordlog.write().await; + + let replica_position_inclusive = if force_commit { + let docs = doc_batch.docs().chain(once(commit_doc())); + mrecordlog_guard + .append_records(&queue_id, None, docs) + .await + .expect("TODO") + } else { + let docs = doc_batch.docs(); + mrecordlog_guard + .append_records(&queue_id, None, docs) + .await + .expect("TODO") + }; + drop(mrecordlog_guard); + + let batch_num_bytes = doc_batch.num_bytes() as u64; + let batch_num_docs = doc_batch.num_docs() as u64; + + INGEST_METRICS + .replicated_num_bytes_total + .inc_by(batch_num_bytes); + INGEST_METRICS + .replicated_num_docs_total + .inc_by(batch_num_docs); + + let replica_shard = state_guard + .replica_shards + .get_mut(&queue_id) + .expect("Replica shard should exist."); + + if replica_position_inclusive != to_position_inclusive { + return Err(IngestV2Error::Internal(format!( + "Bad replica position: expected {to_position_inclusive:?}, got \ + {replica_position_inclusive:?}." + ))); + } + replica_shard.set_replica_position_inclusive(replica_position_inclusive); + + let replicate_success = ReplicateSuccess { + index_uid: subrequest.index_uid, + source_id: subrequest.source_id, + shard_id: subrequest.shard_id, + replica_position_inclusive, + }; + replicate_successes.push(replicate_success); + } + let follower_id = self.follower_id.clone().into(); + let replicate_response = ReplicateResponse { + follower_id, + successes: replicate_successes, + failures: Vec::new(), + }; + Ok(replicate_response) + } + + async fn truncate( + &mut self, + truncate_request: TruncateRequest, + ) -> IngestV2Result { + let mut mrecordlog_guard = self.mrecordlog.write().await; + for truncate_subrequest in truncate_request.subrequests { + let queue_id = truncate_subrequest.queue_id(); + mrecordlog_guard + .truncate(&queue_id, truncate_subrequest.to_position_inclusive) + .await + .expect("TODO"); + } + let truncate_response = TruncateResponse {}; + Ok(truncate_response) + } + + async fn run(&mut self) -> IngestV2Result<()> { + while let Some(syn_replication_message) = self.syn_replication_stream.next().await { + let ack_replication_message = match syn_replication_message.message { + Some(syn_replication_message::Message::ReplicateRequest(replicate_request)) => self + .replicate(replicate_request) + .await + .map(AckReplicationMessage::new_replicate_response), + Some(syn_replication_message::Message::TruncateRequest(truncate_request)) => self + .truncate(truncate_request) + .await + .map(AckReplicationMessage::new_truncate_response), + _ => panic!("TODO"), + }; + if self + .ack_replication_stream_tx + .send(ack_replication_message) + .await + .is_err() + { + break; + } + } + Ok(()) + } +} + +fn into_replication_response(outer_message: AckReplicationMessage) -> Option { + match outer_message.message { + Some(ack_replication_message::Message::ReplicateResponse(replicate_response)) => { + Some(ReplicationResponse::Replicate(replicate_response)) + } + Some(ack_replication_message::Message::TruncateResponse(truncate_response)) => { + Some(ReplicationResponse::Truncate(truncate_response)) + } + _ => None, + } +} + +#[cfg(test)] +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}; + + #[tokio::test] + async fn test_replication_client() { + let (syn_replication_stream_tx, mut syn_replication_stream_rx) = mpsc::channel(5); + let (ack_replication_stream_tx, ack_replication_stream) = ServiceStream::new_bounded(5); + let replication_client = + ReplicationClientTask::spawn(syn_replication_stream_tx, ack_replication_stream); + + let dummy_replication_task_future = async move { + while let Some(sync_replication_message) = syn_replication_stream_rx.recv().await { + let replicate_request = sync_replication_message.into_replicate_request().unwrap(); + let replicate_successes = replicate_request + .subrequests + .iter() + .map(|subrequest| ReplicateSuccess { + index_uid: subrequest.index_uid.clone(), + source_id: subrequest.source_id.clone(), + shard_id: subrequest.shard_id, + replica_position_inclusive: subrequest.to_position_inclusive(), + }) + .collect::>(); + + let replicate_response = ReplicateResponse { + follower_id: replicate_request.follower_id, + successes: replicate_successes, + failures: Vec::new(), + }; + let ack_replication_message = + AckReplicationMessage::new_replicate_response(replicate_response); + ack_replication_stream_tx + .send(Ok(ack_replication_message)) + .await + .unwrap(); + } + }; + tokio::spawn(dummy_replication_task_future); + + let replicate_request = ReplicateRequest { + leader_id: "test-leader".to_string(), + follower_id: "test-follower".to_string(), + commit_type: CommitTypeV2::Auto as i32, + subrequests: vec![ + ReplicateSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + from_position_exclusive: None, + doc_batch: None, + }, + ReplicateSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + from_position_exclusive: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-010"), + doc_lengths: vec![12], + }), + }, + 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], + }), + }, + ], + }; + let replicate_response = replication_client + .replicate(replicate_request) + .await + .unwrap(); + assert_eq!(replicate_response.follower_id, "test-follower"); + assert_eq!(replicate_response.successes.len(), 3); + assert_eq!(replicate_response.failures.len(), 0); + + 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)); + + 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)); + } + + #[tokio::test] + async fn test_replication_task_happy_path() { + let leader_id: NodeId = "test-leader".into(); + let follower_id: NodeId = "test-follower".into(); + let tempdir = tempfile::tempdir().unwrap(); + let mrecordlog = Arc::new(RwLock::new( + MultiRecordLog::open(tempdir.path()).await.unwrap(), + )); + let state = Arc::new(RwLock::new(IngesterState { + primary_shards: HashMap::new(), + replica_shards: HashMap::new(), + replication_clients: HashMap::new(), + replication_tasks: HashMap::new(), + })); + let (syn_replication_stream_tx, syn_replication_stream) = ServiceStream::new_bounded(5); + let (ack_replication_stream_tx, mut ack_replication_stream) = ServiceStream::new_bounded(5); + let _replication_task_handle = ReplicationTask::spawn( + leader_id, + follower_id, + mrecordlog.clone(), + state.clone(), + syn_replication_stream, + ack_replication_stream_tx, + ); + let replicate_request = ReplicateRequest { + leader_id: "test-leader".to_string(), + follower_id: "test-follower".to_string(), + commit_type: CommitTypeV2::Auto as i32, + subrequests: vec![ + ReplicateSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 0, + from_position_exclusive: None, + doc_batch: None, + }, + ReplicateSubrequest { + index_uid: "test-index:0".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + from_position_exclusive: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-010"), + doc_lengths: vec![12], + }), + }, + ReplicateSubrequest { + index_uid: "test-index:1".to_string(), + source_id: "test-source".to_string(), + shard_id: 1, + from_position_exclusive: None, + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-110test-doc-111"), + doc_lengths: vec![12, 12], + }), + }, + ], + }; + let syn_replication_message = + SynReplicationMessage::new_replicate_request(replicate_request); + syn_replication_stream_tx + .send(syn_replication_message) + .await + .unwrap(); + let ack_replication_message = ack_replication_stream.next().await.unwrap().unwrap(); + let replicate_response = into_replication_response(ack_replication_message) + .unwrap() + .into_replicate_response() + .unwrap(); + + assert_eq!(replicate_response.follower_id, "test-follower"); + assert_eq!(replicate_response.successes.len(), 3); + assert_eq!(replicate_response.failures.len(), 0); + + 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_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)); + + 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 mrecordlog_guard = mrecordlog.read().await; + 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); + + mrecordlog_guard.assert_records_eq(&queue_id_00, .., &[]); + + 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); + + mrecordlog_guard.assert_records_eq(&queue_id_01, .., &[(0, "test-doc-010")]); + + 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); + + mrecordlog_guard.assert_records_eq( + &queue_id_11, + .., + &[(0, "test-doc-110"), (1, "test-doc-111")], + ); + drop(mrecordlog_guard); + drop(state_guard); + + let replicate_request = ReplicateRequest { + leader_id: "test-leader".to_string(), + follower_id: "test-follower".to_string(), + commit_type: CommitTypeV2::Auto as i32, + subrequests: vec![ReplicateSubrequest { + 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], + }), + }], + }; + let syn_replication_message = + SynReplicationMessage::new_replicate_request(replicate_request); + syn_replication_stream_tx + .send(syn_replication_message) + .await + .unwrap(); + let ack_replication_message = ack_replication_stream.next().await.unwrap().unwrap(); + let replicate_response = into_replication_response(ack_replication_message) + .unwrap() + .into_replicate_response() + .unwrap(); + + assert_eq!(replicate_response.follower_id, "test-follower"); + assert_eq!(replicate_response.successes.len(), 1); + assert_eq!(replicate_response.failures.len(), 0); + + let replicate_success_0 = &replicate_response.successes[0]; + assert_eq!(replicate_success_0.index_uid, "test-index:0"); + assert_eq!(replicate_success_0.source_id, "test-source"); + assert_eq!(replicate_success_0.shard_id, 1); + assert_eq!(replicate_success_0.replica_position_inclusive, Some(1)); + + let mrecordlog_guard = mrecordlog.read().await; + let state_guard = state.read().await; + + mrecordlog_guard.assert_records_eq( + &queue_id_01, + .., + &[(0, "test-doc-010"), (1, "test-doc-011")], + ); + 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 new file mode 100644 index 00000000000..8204cfd0858 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -0,0 +1,522 @@ +// 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::borrow::Borrow; +use std::collections::HashMap; +use std::fmt; +use std::sync::Arc; + +use async_trait::async_trait; +use futures::stream::FuturesUnordered; +use futures::StreamExt; +use quickwit_proto::control_plane::{ + ControlPlaneService, ControlPlaneServiceClient, GetOpenShardsRequest, GetOpenShardsSubrequest, +}; +use quickwit_proto::ingest::ingester::{IngesterService, PersistRequest, PersistSubrequest}; +use quickwit_proto::ingest::router::{IngestRequestV2, IngestResponseV2, IngestRouterService}; +use quickwit_proto::ingest::IngestV2Result; +use quickwit_proto::types::NodeId; +use quickwit_proto::IndexUid; +use tokio::sync::RwLock; + +use super::shard_table::ShardTable; +use super::{DocBatchBuilderV2, IngesterPool}; + +type LeaderId = String; + +#[derive(Clone)] +pub struct IngestRouter { + self_node_id: NodeId, + control_plane: ControlPlaneServiceClient, + ingester_pool: IngesterPool, + state: Arc>, + replication_factor: usize, +} + +struct RouterState { + shard_table: ShardTable, +} + +impl fmt::Debug for IngestRouter { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("IngestRouter") + .field("self_node_id", &self.self_node_id) + .field("replication_factor", &self.replication_factor) + .finish() + } +} + +impl IngestRouter { + pub fn new( + self_node_id: NodeId, + control_plane: ControlPlaneServiceClient, + ingester_pool: IngesterPool, + replication_factor: usize, + ) -> Self { + let state = RouterState { + shard_table: ShardTable::default(), + }; + Self { + self_node_id, + control_plane, + ingester_pool, + state: Arc::new(RwLock::new(state)), + replication_factor, + } + } + + async fn refresh_shard_table( + &mut self, + ingest_request: &IngestRequestV2, + ) -> IngestV2Result<()> { + let state_guard = self.state.read().await; + + let shard_table = &state_guard.shard_table; + let mut get_open_shards_subrequests = Vec::new(); + + for ingest_subrequest in &ingest_request.subrequests { + if !shard_table + .contains_entry(&*ingest_subrequest.index_id, &ingest_subrequest.source_id) + { + let subrequest = GetOpenShardsSubrequest { + index_id: ingest_subrequest.index_id.clone(), + source_id: ingest_subrequest.source_id.clone(), + }; + get_open_shards_subrequests.push(subrequest); + } + } + if get_open_shards_subrequests.is_empty() { + return Ok(()); + } + drop(state_guard); + + let request = GetOpenShardsRequest { + subrequests: get_open_shards_subrequests, + unavailable_ingesters: Vec::new(), + }; + let response = self.control_plane.get_open_shards(request).await?; + + let mut state_guard = self.state.write().await; + + for subresponse in response.subresponses { + let index_uid: IndexUid = subresponse.index_uid.into(); + let index_id = index_uid.index_id().to_string(); + state_guard.shard_table.update_entry( + index_id, + subresponse.source_id, + subresponse.open_shards, + ); + } + Ok(()) + } +} + +#[async_trait] +impl IngestRouterService for IngestRouter { + async fn ingest( + &mut self, + ingest_request: IngestRequestV2, + ) -> IngestV2Result { + self.refresh_shard_table(&ingest_request).await?; + + let mut doc_batch_builders: Vec = Vec::new(); + let mut persist_subrequests: HashMap<&LeaderId, Vec> = HashMap::new(); + + let state_guard = self.state.read().await; + + // TODO: Here would be the most optimal place to split the body of the HTTP request into + // lines, validate, transform and then pack the docs into compressed batches routed + // to the right shards. + for ingest_subrequest in ingest_request.subrequests { + let table_entry = state_guard + .shard_table + .find_entry(&*ingest_subrequest.index_id, &ingest_subrequest.source_id) + .expect("TODO"); + + if table_entry.len() == 1 { + let shard = &table_entry.shards()[0]; + let persist_subrequest = PersistSubrequest { + index_uid: shard.index_uid.clone(), + source_id: ingest_subrequest.source_id, + shard_id: shard.shard_id, + follower_id: shard.follower_id.clone(), + doc_batch: ingest_subrequest.doc_batch, + }; + persist_subrequests + .entry(&shard.leader_id) + .or_default() + .push(persist_subrequest); + continue; + } + doc_batch_builders.resize_with(table_entry.len(), DocBatchBuilderV2::default); + + for (i, doc) in ingest_subrequest.docs().enumerate() { + let shard_idx = i % table_entry.len(); + doc_batch_builders[shard_idx].add_doc(doc.borrow()); + } + for (shard, doc_batch_builder) in table_entry + .shards() + .iter() + .zip(doc_batch_builders.drain(..)) + { + if !doc_batch_builder.is_empty() { + let doc_batch = doc_batch_builder.build(); + let persist_subrequest = PersistSubrequest { + index_uid: shard.index_uid.clone(), + source_id: ingest_subrequest.source_id.clone(), + shard_id: shard.shard_id, + follower_id: shard.follower_id.clone(), + doc_batch: Some(doc_batch), + }; + persist_subrequests + .entry(&shard.leader_id) + .or_default() + .push(persist_subrequest); + } + } + } + let mut persist_futures = FuturesUnordered::new(); + + for (leader_id, subrequests) in persist_subrequests { + let leader_id: NodeId = leader_id.clone().into(); + let mut ingester = self.ingester_pool.get(&leader_id).await.expect("TODO"); + + let persist_request = PersistRequest { + leader_id: leader_id.into(), + subrequests, + commit_type: ingest_request.commit_type, + }; + let persist_future = async move { ingester.persist(persist_request).await }; + persist_futures.push(persist_future); + } + drop(state_guard); + + while let Some(persist_result) = persist_futures.next().await { + // TODO: Handle errors. + persist_result?; + } + Ok(IngestResponseV2 { + successes: Vec::new(), // TODO + failures: Vec::new(), + }) + } +} + +#[cfg(test)] +mod tests { + use bytes::Bytes; + use quickwit_proto::control_plane::{GetOpenShardsResponse, GetOpenShardsSubresponse}; + use quickwit_proto::ingest::ingester::{IngesterServiceClient, PersistResponse}; + use quickwit_proto::ingest::router::IngestSubrequest; + use quickwit_proto::ingest::{CommitTypeV2, DocBatchV2, Shard}; + + use super::*; + + #[tokio::test] + async fn test_router_refresh_shard_table() { + let self_node_id = "test-router".into(); + + let mut control_plane_mock = ControlPlaneServiceClient::mock(); + control_plane_mock + .expect_get_open_shards() + .once() + .returning(|request| { + assert_eq!(request.subrequests.len(), 2); + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.index_id, "test-index-0"); + assert_eq!(subrequest.source_id, "test-source"); + + let subrequest = &request.subrequests[1]; + assert_eq!(subrequest.index_id, "test-index-1"); + assert_eq!(subrequest.source_id, "test-source"); + + let response = GetOpenShardsResponse { + subresponses: vec![GetOpenShardsSubresponse { + index_uid: "test-index-0:0".to_string(), + source_id: "test-source".to_string(), + open_shards: vec![Shard { + shard_id: 0, + ..Default::default() + }], + }], + }; + Ok(response) + }); + let control_plane: ControlPlaneServiceClient = control_plane_mock.into(); + let ingester_pool = IngesterPool::default(); + let replication_factor = 1; + let mut router = IngestRouter::new( + self_node_id, + control_plane, + ingester_pool.clone(), + replication_factor, + ); + + let ingest_request = IngestRequestV2 { + subrequests: Vec::new(), + commit_type: CommitTypeV2::Auto as i32, + }; + router.refresh_shard_table(&ingest_request).await.unwrap(); + assert!(router.state.read().await.shard_table.is_empty()); + + let ingest_request = IngestRequestV2 { + subrequests: vec![ + IngestSubrequest { + index_id: "test-index-0".to_string(), + source_id: "test-source".to_string(), + ..Default::default() + }, + IngestSubrequest { + index_id: "test-index-1".to_string(), + source_id: "test-source".to_string(), + ..Default::default() + }, + ], + commit_type: CommitTypeV2::Auto as i32, + }; + router.refresh_shard_table(&ingest_request).await.unwrap(); + + let state_guard = router.state.read().await; + let shard_table = &state_guard.shard_table; + assert_eq!(shard_table.len(), 1); + + let routing_entry_0 = shard_table + .find_entry("test-index-0", "test-source") + .unwrap(); + assert_eq!(routing_entry_0.len(), 1); + assert_eq!(routing_entry_0.shards()[0].shard_id, 0); + drop(state_guard); + + let mut control_plane_mock = ControlPlaneServiceClient::mock(); + control_plane_mock + .expect_get_open_shards() + .once() + .returning(|request| { + assert_eq!(request.subrequests.len(), 1); + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.index_id, "test-index-1"); + assert_eq!(subrequest.source_id, "test-source"); + + let response = GetOpenShardsResponse { + subresponses: vec![GetOpenShardsSubresponse { + index_uid: "test-index-1:1".to_string(), + source_id: "test-source".to_string(), + open_shards: vec![ + Shard { + shard_id: 0, + ..Default::default() + }, + Shard { + shard_id: 1, + ..Default::default() + }, + ], + }], + }; + Ok(response) + }); + let control_plane: ControlPlaneServiceClient = control_plane_mock.into(); + router.control_plane = control_plane; + + let ingest_request = IngestRequestV2 { + subrequests: vec![ + IngestSubrequest { + index_id: "test-index-0".to_string(), + source_id: "test-source".to_string(), + ..Default::default() + }, + IngestSubrequest { + index_id: "test-index-1".to_string(), + source_id: "test-source".to_string(), + ..Default::default() + }, + ], + commit_type: CommitTypeV2::Auto as i32, + }; + router.refresh_shard_table(&ingest_request).await.unwrap(); + + let state_guard = router.state.read().await; + let shard_table = &state_guard.shard_table; + assert_eq!(shard_table.len(), 2); + + let routing_entry_0 = shard_table + .find_entry("test-index-0", "test-source") + .unwrap(); + assert_eq!(routing_entry_0.len(), 1); + assert_eq!(routing_entry_0.shards()[0].shard_id, 0); + + let routing_entry_1 = shard_table + .find_entry("test-index-1", "test-source") + .unwrap(); + assert_eq!(routing_entry_1.len(), 2); + assert_eq!(routing_entry_1.shards()[0].shard_id, 0); + assert_eq!(routing_entry_1.shards()[1].shard_id, 1); + } + + #[tokio::test] + async fn test_router_ingest() { + let self_node_id = "test-router".into(); + let control_plane = ControlPlaneServiceClient::mock().into(); + let ingester_pool = IngesterPool::default(); + let replication_factor = 1; + let mut router = IngestRouter::new( + self_node_id, + control_plane, + ingester_pool.clone(), + replication_factor, + ); + + let mut state_guard = router.state.write().await; + state_guard.shard_table.update_entry( + "test-index-0", + "test-source", + vec![Shard { + index_uid: "test-index-0:0".to_string(), + shard_id: 0, + leader_id: "test-ingester-0".to_string(), + ..Default::default() + }], + ); + state_guard.shard_table.update_entry( + "test-index-1", + "test-source", + vec![ + Shard { + index_uid: "test-index-1:1".to_string(), + shard_id: 0, + leader_id: "test-ingester-0".to_string(), + follower_id: Some("test-ingester-1".to_string()), + ..Default::default() + }, + Shard { + index_uid: "test-index-1:1".to_string(), + shard_id: 1, + leader_id: "test-ingester-1".to_string(), + follower_id: Some("test-ingester-2".to_string()), + ..Default::default() + }, + ], + ); + drop(state_guard); + + let mut ingester_mock_0 = IngesterServiceClient::mock(); + ingester_mock_0 + .expect_persist() + .once() + .returning(|mut request| { + assert_eq!(request.leader_id, "test-ingester-0"); + assert_eq!(request.subrequests.len(), 2); + assert_eq!(request.commit_type, CommitTypeV2::Auto as i32); + + request + .subrequests + .sort_unstable_by(|left, right| left.index_uid.cmp(&right.index_uid)); + + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.index_uid, "test-index-0:0"); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.shard_id, 0); + assert_eq!(subrequest.follower_id(), ""); + assert_eq!( + subrequest.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-000test-doc-001" + ); + assert_eq!(subrequest.doc_batch.as_ref().unwrap().doc_lengths, [12, 12]); + + let subrequest = &request.subrequests[1]; + assert_eq!(subrequest.index_uid, "test-index-1:1"); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.shard_id, 0); + assert_eq!(subrequest.follower_id(), "test-ingester-1"); + assert_eq!( + subrequest.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-100test-doc-102" + ); + assert_eq!(subrequest.doc_batch.as_ref().unwrap().doc_lengths, [12, 12]); + + let response = PersistResponse { + leader_id: request.leader_id, + successes: Vec::new(), + failures: Vec::new(), + }; + Ok(response) + }); + let ingester_0: IngesterServiceClient = ingester_mock_0.into(); + ingester_pool + .insert("test-ingester-0".into(), ingester_0.clone()) + .await; + + let mut ingester_mock_1 = IngesterServiceClient::mock(); + ingester_mock_1 + .expect_persist() + .once() + .returning(|request| { + assert_eq!(request.leader_id, "test-ingester-1"); + assert_eq!(request.subrequests.len(), 1); + assert_eq!(request.commit_type, CommitTypeV2::Auto as i32); + + let subrequest = &request.subrequests[0]; + assert_eq!(subrequest.index_uid, "test-index-1:1"); + assert_eq!(subrequest.source_id, "test-source"); + assert_eq!(subrequest.shard_id, 1); + assert_eq!(subrequest.follower_id(), "test-ingester-2"); + assert_eq!( + subrequest.doc_batch.as_ref().unwrap().doc_buffer, + "test-doc-111test-doc-113" + ); + assert_eq!(subrequest.doc_batch.as_ref().unwrap().doc_lengths, [12, 12]); + + let response = PersistResponse { + leader_id: request.leader_id, + successes: Vec::new(), + failures: Vec::new(), + }; + Ok(response) + }); + let ingester_1: IngesterServiceClient = ingester_mock_1.into(); + ingester_pool + .insert("test-ingester-1".into(), ingester_1) + .await; + + let ingest_request = IngestRequestV2 { + commit_type: CommitTypeV2::Auto as i32, + subrequests: vec![ + IngestSubrequest { + index_id: "test-index-0".to_string(), + source_id: "test-source".to_string(), + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc-000test-doc-001"), + doc_lengths: vec![12, 12], + }), + }, + IngestSubrequest { + index_id: "test-index-1".to_string(), + source_id: "test-source".to_string(), + doc_batch: Some(DocBatchV2 { + doc_buffer: Bytes::from_static( + b"test-doc-100test-doc-111test-doc-102test-doc-113", + ), + doc_lengths: vec![12, 12, 12, 12], + }), + }, + ], + }; + router.ingest(ingest_request).await.unwrap(); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs new file mode 100644 index 00000000000..aeb90c7d77d --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/shard_table.rs @@ -0,0 +1,144 @@ +// 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::collections::HashMap; + +use quickwit_proto::ingest::Shard; +use quickwit_proto::types::SourceId; +use quickwit_proto::IndexId; + +/// A set of open shards for a given index and source. +#[derive(Debug, Default)] +pub(crate) struct ShardTableEntry { + shards: Vec, +} + +impl ShardTableEntry { + /// Creates a new entry and ensures that the shards are open and unique. + pub fn new(mut shards: Vec) -> Self { + shards.retain(|shard| shard.is_open()); + shards.sort_unstable_by_key(|shard| shard.shard_id); + shards.dedup_by_key(|shard| shard.shard_id); + + Self { shards } + } + + /// Returns the number of shards that make up the entry. + pub fn len(&self) -> usize { + self.shards.len() + } + + /// Returns the shards that make up the entry. + pub fn shards(&self) -> &[Shard] { + &self.shards + } +} + +/// A table of shard entries indexed by index UID and source ID. +#[derive(Debug, Default)] +pub(crate) struct ShardTable { + table: HashMap<(IndexId, SourceId), ShardTableEntry>, +} + +impl ShardTable { + pub fn contains_entry( + &self, + index_id: impl Into, + source_id: impl Into, + ) -> bool { + let key = (index_id.into(), source_id.into()); + self.table.contains_key(&key) + } + + pub fn find_entry( + &self, + index_id: impl Into, + source_id: impl Into, + ) -> Option<&ShardTableEntry> { + let key = (index_id.into(), source_id.into()); + self.table.get(&key) + } + + pub fn update_entry( + &mut self, + index_id: impl Into, + source_id: impl Into, + shards: Vec, + ) { + let key = (index_id.into(), source_id.into()); + self.table.insert(key, ShardTableEntry::new(shards)); + } + + #[cfg(test)] + pub fn is_empty(&self) -> bool { + self.table.is_empty() + } + + #[cfg(test)] + pub fn len(&self) -> usize { + self.table.len() + } +} + +#[cfg(test)] +mod tests { + use quickwit_proto::ingest::ShardState; + + use super::*; + + #[test] + fn test_shard_table() { + let mut table = ShardTable::default(); + assert!(!table.contains_entry("test-index", "test-source")); + + table.update_entry( + "test-index", + "test-source", + vec![ + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 0, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 1, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 0, + ..Default::default() + }, + Shard { + index_uid: "test-index:0".to_string(), + shard_id: 2, + shard_state: ShardState::Closed as i32, + ..Default::default() + }, + ], + ); + assert!(table.contains_entry("test-index", "test-source")); + + let entry = table.find_entry("test-index", "test-source").unwrap(); + assert_eq!(entry.len(), 2); + assert_eq!(entry.shards()[0].shard_id, 0); + assert_eq!(entry.shards()[1].shard_id, 1); + } +} diff --git a/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs b/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs new file mode 100644 index 00000000000..e013354f383 --- /dev/null +++ b/quickwit/quickwit-ingest/src/ingest_v2/test_utils.rs @@ -0,0 +1,146 @@ +// 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::ops::RangeBounds; + +use mrecordlog::MultiRecordLog; +use quickwit_proto::ingest::ShardState; + +use super::ingester::{PrimaryShard, ReplicaShard}; +use crate::ingest_v2::ingester::Position; + +pub(super) trait MultiRecordLogTestExt { + fn assert_records_eq(&self, queue_id: &str, range: R, expected_records: &[(u64, &str)]) + where R: RangeBounds + 'static; +} + +impl MultiRecordLogTestExt for MultiRecordLog { + #[track_caller] + fn assert_records_eq(&self, queue_id: &str, range: R, expected_records: &[(u64, &str)]) + where R: RangeBounds + 'static { + let records = self + .range(queue_id, range) + .unwrap() + .map(|(position, record)| (position, String::from_utf8(record.into_owned()).unwrap())) + .collect::>(); + assert_eq!( + records.len(), + expected_records.len(), + "expected {} records, got {}", + expected_records.len(), + records.len() + ); + for ((position, record), (expected_position, expected_record)) in + records.iter().zip(expected_records.iter()) + { + assert_eq!( + position, expected_position, + "expected record at position `{expected_position}`, got `{position}`", + ); + assert_eq!( + record, expected_record, + "expected record `{expected_record}`, got `{record}`", + ); + } + } +} + +pub(super) trait PrimaryShardTestExt { + fn assert_positions( + &self, + expected_primary_position: impl Into, + expected_replica_position: Option>, + ); + + 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_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-ingest/src/lib.rs b/quickwit/quickwit-ingest/src/lib.rs index 1d992bc7718..678007293ce 100644 --- a/quickwit/quickwit-ingest/src/lib.rs +++ b/quickwit/quickwit-ingest/src/lib.rs @@ -19,10 +19,12 @@ #![deny(clippy::disallowed_methods)] +mod doc_batch; mod errors; mod ingest_api_service; #[path = "codegen/ingest_service.rs"] mod ingest_service; +mod ingest_v2; mod memory_capacity; mod metrics; mod notifications; @@ -33,9 +35,11 @@ use std::collections::HashMap; use std::path::{Path, PathBuf}; use anyhow::{bail, Context}; +pub use doc_batch::*; pub use errors::IngestServiceError; pub use ingest_api_service::{GetMemoryCapacity, GetPartitionId, IngestApiService}; pub use ingest_service::*; +pub use ingest_v2::*; pub use memory_capacity::MemoryCapacity; use once_cell::sync::OnceCell; pub use position::Position; @@ -44,9 +48,6 @@ use quickwit_actors::{Mailbox, Universe}; use quickwit_config::IngestApiConfig; use tokio::sync::Mutex; -mod doc_batch; -pub use doc_batch::*; - pub const QUEUES_DIR_NAME: &str = "queues"; pub type Result = std::result::Result; diff --git a/quickwit/quickwit-ingest/src/metrics.rs b/quickwit/quickwit-ingest/src/metrics.rs index 0c185221e71..d73220c8474 100644 --- a/quickwit/quickwit-ingest/src/metrics.rs +++ b/quickwit/quickwit-ingest/src/metrics.rs @@ -23,6 +23,8 @@ use quickwit_common::metrics::{new_counter, new_gauge, IntCounter, IntGauge}; pub struct IngestMetrics { pub ingested_num_bytes: IntCounter, pub ingested_num_docs: IntCounter, + pub replicated_num_bytes_total: IntCounter, + pub replicated_num_docs_total: IntCounter, pub queue_count: IntGauge, } @@ -39,6 +41,16 @@ impl Default for IngestMetrics { "Number of docs received to be ingested", "quickwit_ingest", ), + replicated_num_bytes_total: new_counter( + "replicated_num_bytes_total", + "Total size in bytes of the replicated docs.", + "quickwit_ingest", + ), + replicated_num_docs_total: new_counter( + "replicated_num_docs_total", + "Total number of docs replicated.", + "quickwit_ingest", + ), queue_count: new_gauge( "queue_count", "Number of queues currently active", diff --git a/quickwit/quickwit-jaeger/src/lib.rs b/quickwit/quickwit-jaeger/src/lib.rs index d1e0b8ef54a..c12debdc5bc 100644 --- a/quickwit/quickwit-jaeger/src/lib.rs +++ b/quickwit/quickwit-jaeger/src/lib.rs @@ -43,7 +43,7 @@ use quickwit_proto::jaeger::storage::v1::{ SpansResponseChunk, TraceQueryParameters, }; use quickwit_proto::opentelemetry::proto::trace::v1::status::StatusCode as OtlpStatusCode; -use quickwit_proto::{ListTermsRequest, SearchRequest}; +use quickwit_proto::search::{ListTermsRequest, SearchRequest}; use quickwit_query::query_ast::{BoolQuery, QueryAst, RangeQuery, TermQuery}; use quickwit_search::{FindTraceIdsCollector, SearchService}; use serde::Deserialize; @@ -2381,7 +2381,7 @@ mod tests { && req.start_timestamp.is_some() }) .return_once(|_| { - Ok(quickwit_proto::ListTermsResponse { + Ok(quickwit_proto::search::ListTermsResponse { num_hits: 3, terms: vec![ encode_term_for_test!("service1"), diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index 13c442e2017..d22024b171f 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -292,7 +292,7 @@ mod tests { use quickwit_indexing::TestSandbox; use quickwit_metastore::SplitState; use quickwit_proto::metastore::DeleteQuery; - use quickwit_proto::{LeafSearchRequest, LeafSearchResponse}; + use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{ searcher_pool_for_test, MockSearchService, SearchError, SearchJobPlacer, }; diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs index d775a61cb5e..4b49afd5af7 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs @@ -33,7 +33,8 @@ use quickwit_metastore::{ split_tag_filter, split_time_range_filter, Metastore, MetastoreResult, Split, }; use quickwit_proto::metastore::DeleteTask; -use quickwit_proto::{IndexUid, SearchRequest}; +use quickwit_proto::search::SearchRequest; +use quickwit_proto::IndexUid; use quickwit_search::{jobs_to_leaf_request, SearchJob, SearchJobPlacer}; use serde::Serialize; use tantivy::Inventory; @@ -401,7 +402,7 @@ mod tests { use quickwit_indexing::TestSandbox; use quickwit_metastore::SplitMetadata; use quickwit_proto::metastore::DeleteQuery; - use quickwit_proto::{LeafSearchRequest, LeafSearchResponse}; + use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{searcher_pool_for_test, MockSearchService}; use tantivy::TrackedObject; diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs index 2d0094ec0b8..db8c0246e26 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed_metastore/file_backed_index/serialize.rs @@ -79,7 +79,7 @@ impl From for FileBackedIndex { fn from(mut index: FileBackedIndexV0_6) -> Self { // Override split index_id to support old SplitMetadata format. for split in index.splits.iter_mut() { - if split.split_metadata.index_uid.to_string().is_empty() { + if split.split_metadata.index_uid.is_empty() { split.split_metadata.index_uid = index.metadata.index_uid.clone(); } } diff --git a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs index da1c4c10427..46f82e2394d 100644 --- a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs +++ b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/grpc_adapter.rs @@ -22,7 +22,6 @@ use std::sync::Arc; use async_trait::async_trait; use itertools::Itertools; use quickwit_config::IndexConfig; -use quickwit_proto::metastore::metastore_service_server::{self as grpc}; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteIndexRequest, DeleteIndexResponse, DeleteQuery, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, @@ -30,8 +29,8 @@ use quickwit_proto::metastore::{ LastDeleteOpstampResponse, ListAllSplitsRequest, ListDeleteTasksRequest, ListDeleteTasksResponse, ListIndexesMetadatasRequest, ListIndexesMetadatasResponse, ListSplitsRequest, ListSplitsResponse, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, - PublishSplitsRequest, ResetSourceCheckpointRequest, SourceResponse, SplitResponse, - StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, + MetastoreService, PublishSplitsRequest, ResetSourceCheckpointRequest, SourceResponse, + SplitResponse, StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::tonic::{Request, Response, Status}; @@ -51,7 +50,7 @@ impl From> for GrpcMetastoreAdapter { } #[async_trait] -impl grpc::MetastoreService for GrpcMetastoreAdapter { +impl MetastoreService for GrpcMetastoreAdapter { #[instrument(skip(self, request))] async fn create_index( &self, @@ -206,7 +205,7 @@ impl grpc::MetastoreService for GrpcMetastoreAdapter { set_parent_span_from_request_metadata(request.metadata()); let publish_request = request.into_inner(); let split_ids = publish_request - .split_ids + .staged_split_ids .iter() .map(|split_id| split_id.as_str()) .collect_vec(); diff --git a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs index 59b0c9a573a..5b98e2ff396 100644 --- a/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/grpc_metastore/mod.rs @@ -29,13 +29,12 @@ use itertools::Itertools; use quickwit_common::tower::BalanceChannel; use quickwit_common::uri::Uri as QuickwitUri; use quickwit_config::{IndexConfig, SourceConfig}; -use quickwit_proto::metastore::metastore_service_client::MetastoreServiceClient; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, DeleteQuery, DeleteSourceRequest, DeleteSplitsRequest, DeleteTask, IndexMetadataRequest, LastDeleteOpstampRequest, ListAllSplitsRequest, ListDeleteTasksRequest, ListIndexesMetadatasRequest, ListSplitsRequest, - ListStaleSplitsRequest, MarkSplitsForDeletionRequest, PublishSplitsRequest, - ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, + ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreServiceClient, + PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, UpdateSplitsDeleteOpstampRequest, }; use quickwit_proto::tonic::codegen::InterceptedService; @@ -241,7 +240,8 @@ impl Metastore for MetastoreGrpcClient { replaced_split_ids: &[&'a str], checkpoint_delta_opt: Option, ) -> MetastoreResult<()> { - let split_ids_vec: Vec = split_ids.iter().map(|split| split.to_string()).collect(); + let staged_split_ids: Vec = + split_ids.iter().map(|split| split.to_string()).collect(); let replaced_split_ids_vec: Vec = replaced_split_ids .iter() .map(|split_id| split_id.to_string()) @@ -255,7 +255,7 @@ impl Metastore for MetastoreGrpcClient { })?; let request = PublishSplitsRequest { index_uid: index_uid.into(), - split_ids: split_ids_vec, + staged_split_ids, replaced_split_ids: replaced_split_ids_vec, index_checkpoint_delta_serialized_json, }; @@ -569,7 +569,7 @@ impl crate::tests::test_suite::DefaultForTest for MetastoreGrpcClient { async fn default_for_test() -> Self { use std::sync::Arc; - use quickwit_proto::metastore::metastore_service_server::MetastoreServiceServer; + use quickwit_proto::metastore::MetastoreServiceServer; use quickwit_proto::tonic::transport::Server; use quickwit_storage::RamStorage; diff --git a/quickwit/quickwit-proto/Cargo.toml b/quickwit/quickwit-proto/Cargo.toml index 06cebb3f436..2e7eb9c3ec2 100644 --- a/quickwit/quickwit-proto/Cargo.toml +++ b/quickwit/quickwit-proto/Cargo.toml @@ -12,6 +12,7 @@ documentation = "https://quickwit.io/docs/" [dependencies] anyhow = { workspace = true } async-trait = { workspace = true } +bytes = { 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 c6f3caf525b..efb87666651 100644 --- a/quickwit/quickwit-proto/build.rs +++ b/quickwit/quickwit-proto/build.rs @@ -23,60 +23,82 @@ use glob::glob; use quickwit_codegen::Codegen; fn main() -> Result<(), Box> { - // "Classic" prost + tonic codegen for metastore and search services. - let protos: Vec = find_protos("protos/quickwit") - .into_iter() - .filter(|path| !path.ends_with("control_pane.proto") || !path.ends_with("indexing.proto")) - .collect(); - - let mut prost_config = prost_build::Config::default(); - prost_config.protoc_arg("--experimental_allow_proto3_optional"); - - tonic_build::configure() - .type_attribute(".", "#[derive(Serialize, Deserialize, utoipa::ToSchema)]") - .type_attribute("SearchRequest", "#[derive(Eq, Hash)]") - .type_attribute("SortField", "#[derive(Eq, Hash)]") - .type_attribute("SortByValue", "#[derive(Ord, PartialOrd)]") - .type_attribute("DeleteQuery", "#[serde(default)]") - .field_attribute( - "DeleteQuery.start_timestamp", - "#[serde(skip_serializing_if = \"Option::is_none\")]", - ) - .field_attribute( - "DeleteQuery.end_timestamp", - "#[serde(skip_serializing_if = \"Option::is_none\")]", - ) - .type_attribute("PartialHit.sort_value", "#[derive(Copy)]") - .enum_attribute(".", "#[serde(rename_all=\"snake_case\")]") - .out_dir("src/codegen/quickwit") - .compile_with_config(prost_config, &protos, &["protos/quickwit"])?; - - // Prost + tonic + Quickwit codegen for control plane and indexing services. + // Prost + tonic + Quickwit codegen for control plane, indexing, and ingest services. // // Control plane Codegen::run( &["protos/quickwit/control_plane.proto"], "src/codegen/quickwit", - "crate::control_plane::Result", + "crate::control_plane::ControlPlaneResult", "crate::control_plane::ControlPlaneError", - &[], + &["protos"], ) .unwrap(); // Indexing Service - let mut index_api_config = prost_build::Config::default(); - index_api_config.type_attribute("IndexingTask", "#[derive(Eq, Hash)]"); + let mut prost_config = prost_build::Config::default(); + prost_config.type_attribute("IndexingTask", "#[derive(Eq, Hash)]"); Codegen::run_with_config( &["protos/quickwit/indexing.proto"], "src/codegen/quickwit", - "crate::indexing::Result", + "crate::indexing::IndexingResult", "crate::indexing::IndexingError", &[], - index_api_config, + prost_config, + ) + .unwrap(); + + // Ingest service + let mut prost_config = prost_build::Config::default(); + prost_config.bytes(["DocBatchV2.doc_buffer"]); + + Codegen::run_with_config( + &[ + "protos/quickwit/ingester.proto", + "protos/quickwit/router.proto", + ], + "src/codegen/quickwit", + "crate::ingest::IngestV2Result", + "crate::ingest::IngestV2Error", + &["protos"], + prost_config, ) .unwrap(); + // "Classic" prost + tonic codegen for metastore and search services. + let mut prost_config = prost_build::Config::default(); + prost_config + .bytes(["DocBatchV2.doc_buffer"]) + .protoc_arg("--experimental_allow_proto3_optional"); + + tonic_build::configure() + .enum_attribute(".", "#[serde(rename_all=\"snake_case\")]") + .field_attribute("DeleteQuery.index_uid", "#[serde(alias = \"index_id\")]") + .field_attribute("DeleteQuery.query_ast", "#[serde(alias = \"query\")]") + .field_attribute( + "DeleteQuery.start_timestamp", + "#[serde(skip_serializing_if = \"Option::is_none\")]", + ) + .field_attribute( + "DeleteQuery.end_timestamp", + "#[serde(skip_serializing_if = \"Option::is_none\")]", + ) + .type_attribute(".", "#[derive(Serialize, Deserialize, utoipa::ToSchema)]") + .type_attribute("PartialHit.sort_value", "#[derive(Copy)]") + .type_attribute("SearchRequest", "#[derive(Eq, Hash)]") + .type_attribute("SortByValue", "#[derive(Ord, PartialOrd)]") + .type_attribute("SortField", "#[derive(Eq, Hash)]") + .out_dir("src/codegen/quickwit") + .compile_with_config( + prost_config, + &[ + "protos/quickwit/metastore.proto", + "protos/quickwit/search.proto", + ], + &["protos"], + )?; + // Jaeger proto let protos = find_protos("protos/third-party/jaeger"); diff --git a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto index 400a532496b..f8829c85f5a 100644 --- a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto +++ b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto @@ -21,6 +21,9 @@ syntax = "proto3"; package quickwit.control_plane; +import "quickwit/ingest.proto"; +import "quickwit/metastore.proto"; + service ControlPlaneService { /// Notify the Control Plane that a change on an index occurred. The change /// can be an index creation, deletion, or update that includes a source creation/deletion/num pipeline update. @@ -28,9 +31,54 @@ service ControlPlaneService { // This can be explicited in the attributes of `NotifyIndexChangeRequest` with an enum that describes the // type of change. The index ID and/or source ID could also be added. // However, these attributes will not be used by the Control Plane, at least at short term. - rpc notifyIndexChange(NotifyIndexChangeRequest) returns (NotifyIndexChangeResponse); + rpc NotifyIndexChange(NotifyIndexChangeRequest) returns (NotifyIndexChangeResponse); + + /// 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 GetOpenShards(GetOpenShardsRequest) returns (GetOpenShardsResponse); + + rpc CloseShards(CloseShardsRequest) returns (CloseShardsResponse); } message NotifyIndexChangeRequest {} message NotifyIndexChangeResponse {} + +/// +/// Shard API +/// + +message GetOpenShardsRequest { + repeated GetOpenShardsSubrequest subrequests = 1; + repeated string unavailable_ingesters = 2; +} + +message GetOpenShardsSubrequest { + string index_id = 1; + string source_id = 2; +} + +message GetOpenShardsResponse { + repeated GetOpenShardsSubresponse subresponses = 1; +} + +message GetOpenShardsSubresponse { + string index_uid = 1; + string source_id = 2; + repeated quickwit.ingest.Shard open_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 { +} diff --git a/quickwit/quickwit-proto/protos/quickwit/indexing.proto b/quickwit/quickwit-proto/protos/quickwit/indexing.proto index d80ddcdc63a..3cb778cead3 100644 --- a/quickwit/quickwit-proto/protos/quickwit/indexing.proto +++ b/quickwit/quickwit-proto/protos/quickwit/indexing.proto @@ -23,18 +23,20 @@ package quickwit.indexing; service IndexingService { /// Apply an indexing plan on the node. - rpc applyIndexingPlan(ApplyIndexingPlanRequest) returns (ApplyIndexingPlanResponse); + rpc ApplyIndexingPlan(ApplyIndexingPlanRequest) returns (ApplyIndexingPlanResponse); } message ApplyIndexingPlanRequest { repeated IndexingTask indexing_tasks = 1; } -message ApplyIndexingPlanResponse {} - message IndexingTask { - /// Index UID of the task. + /// The tasks's index UID. string index_uid = 1; - /// Source ID of the task. + /// The task's source ID. string source_id = 2; + /// The shards assigned to the indexer. + // repeated uint64 shard_ids = 3; } + +message ApplyIndexingPlanResponse {} diff --git a/quickwit/quickwit-proto/protos/quickwit/ingest.proto b/quickwit/quickwit-proto/protos/quickwit/ingest.proto new file mode 100644 index 00000000000..93b287d2872 --- /dev/null +++ b/quickwit/quickwit-proto/protos/quickwit/ingest.proto @@ -0,0 +1,65 @@ +// 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 . + +syntax = "proto3"; + +package quickwit.ingest; + + +enum CommitTypeV2 { + Auto = 0; + Wait = 1; + Force = 2; +} + +message DocBatchV2 { + bytes doc_buffer = 1; + repeated uint32 doc_lengths = 2; +} + +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 the its leader or follower to close it with its final replication position, after which write requests will be rejected. + CLOSING = 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; +} + +message Shard { + // Immutable fields + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + /// The node ID of the ingester to which all the write requests for this shard should be sent to. + string leader_id = 4; + /// The node ID of the ingester holding a copy of the data. + optional string follower_id = 5; + + // 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; + optional string publish_token = 11; +} diff --git a/quickwit/quickwit-proto/protos/quickwit/ingester.proto b/quickwit/quickwit-proto/protos/quickwit/ingester.proto new file mode 100644 index 00000000000..b7817a006ce --- /dev/null +++ b/quickwit/quickwit-proto/protos/quickwit/ingester.proto @@ -0,0 +1,180 @@ +// 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 . + +syntax = "proto3"; + +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); + + /// Opens a replication stream from a leader to a follower. + rpc OpenReplicationStream(stream SynReplicationMessage) returns (stream AckReplicationMessage); + + /// Streams records from a leader or a follower. The client can optionally specify a range of positions to fetch. + rpc OpenFetchStream(OpenFetchStreamRequest) returns (stream FetchResponseV2); + + // rpc OpenWatchStream(OpenWatchStreamRequest) returns (stream WatchMessage); + + /// Pings an ingester to check if it is ready to host shards and serve requests. + rpc Ping(PingRequest) returns (PingResponse); + + /// Truncates the shards at the given positions. Callers should this RPC on leaders and they will forward to followers. + rpc Truncate(TruncateRequest) returns (TruncateResponse); +} + +message PersistRequest { + string leader_id = 1; + quickwit.ingest.CommitTypeV2 commit_type = 3; + repeated PersistSubrequest subrequests = 4; +} + +message PersistSubrequest { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + optional string follower_id = 4; + quickwit.ingest.DocBatchV2 doc_batch = 5; +} + +message PersistResponse { + string leader_id = 1; + repeated PersistSuccess successes = 2; + repeated PersistFailure failures = 3; +} + +message PersistSuccess { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + optional uint64 replication_position_inclusive = 4; +} + +message PersistFailure { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + // ingest.DocBatchV2 doc_batch = 4; + // ingest.IngestError error = 5; +} + +message SynReplicationMessage { + oneof message { + OpenReplicationStreamRequest open_request = 1; + ReplicateRequest replicate_request = 2; + TruncateRequest truncate_request = 3; + } +} + +message AckReplicationMessage { + oneof message { + OpenReplicationStreamResponse open_response = 1; + ReplicateResponse replicate_response = 3; + TruncateResponse truncate_response = 4; + } +} + +message OpenReplicationStreamRequest { + string leader_id = 1; + string follower_id = 2; +} + +message OpenReplicationStreamResponse { +} + +message ReplicateRequest { + string leader_id = 1; + string follower_id = 2; + quickwit.ingest.CommitTypeV2 commit_type = 3; + repeated ReplicateSubrequest subrequests = 4; +} + +message ReplicateSubrequest { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + optional uint64 from_position_exclusive = 4; + ingest.DocBatchV2 doc_batch = 6; +} + +message ReplicateResponse { + string follower_id = 1; + repeated ReplicateSuccess successes = 2; + repeated ReplicateFailure failures = 3; +} + +message ReplicateSuccess { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + optional uint64 replica_position_inclusive = 4; +} + +message ReplicateFailure { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + // ingest.DocBatchV2 doc_batch = 4; + // ingest.IngestError error = 5; +} + +message TruncateRequest { + string leader_id = 1; + repeated TruncateSubrequest subrequests = 2; +} + +message TruncateSubrequest { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + uint64 to_position_inclusive = 4; +} + +message TruncateResponse { + // TODO +} + +message OpenFetchStreamRequest { + string client_id = 1; + string index_uid = 2; + string source_id = 3; + uint64 shard_id = 4; + optional uint64 from_position_exclusive = 5; + optional uint64 to_position_inclusive = 6; +} + +message FetchResponseV2 { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + uint64 from_position_inclusive = 4; + ingest.DocBatchV2 doc_batch = 5; +} + +message PingRequest { + string leader_id = 1; + optional string follower_id = 2; +} + +message PingResponse { +} diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 03c263b6571..d66ad0b5228 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -22,6 +22,8 @@ syntax = "proto3"; package quickwit.metastore; +import "quickwit/ingest.proto"; + service MetastoreService { // Creates an index. rpc create_index(CreateIndexRequest) returns (CreateIndexResponse); @@ -79,6 +81,18 @@ service MetastoreService { /// Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_id`. rpc list_stale_splits(ListStaleSplitsRequest) returns (ListSplitsResponse); + + /// + /// Shard API + /// + + // rpc OpenShards(OpenShardsRequest) returns (OpenShardsResponse); + + // rpc CloseShards(CloseShardsRequest) returns (CloseShardsResponse); + + // rpc DeleteShards(DeleteShardsRequest) returns (DeleteShardsResponse); + + // rpc ListShards(ListShardsRequest) returns (ListShardsResponse); } message CreateIndexRequest { @@ -130,9 +144,10 @@ message StageSplitsRequest { message PublishSplitsRequest { string index_uid = 1; - repeated string split_ids = 2; + repeated string staged_split_ids = 2; repeated string replaced_split_ids = 3; optional string index_checkpoint_delta_serialized_json = 4; + // optional string publish_token = 5; } message MarkSplitsForDeletionRequest { @@ -226,3 +241,93 @@ message ListDeleteTasksResponse { repeated DeleteTask delete_tasks = 1; } +/// +/// Shard API +/// + +// message OpenShardsRequest { +// repeated OpenShardsSubrequest subrequests = 1; +// } + +// message OpenShardsSubrequest { +// string index_uid = 1; +// string source_id = 2; +// repeated uint64 shard_ids = 3; +// string leader_id = 4; +// optional string follower_id = 5; +// } + +// message OpenShardsResponse { +// repeated OpenShardsSubresponse subresponses = 1; +// } + +// message OpenShardsSubresponse { +// string index_uid = 1; +// string source_id = 2; +// repeated quickwit.ingest.Shard open_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; +// } + +// message CloseShardsFailure { +// string index_uid = 1; +// string source_id = 2; +// uint64 shard_id = 3; +// uint32 error_code = 4; +// string error_message = 5; +// } + +// message DeleteShardsRequest { +// repeated DeleteShardsSubrequest subrequests = 1; +// bool force = 2; +// } + +// message DeleteShardsSubrequest { +// string index_uid = 1; +// string source_id = 2; +// repeated uint64 shard_ids = 3; +// } + +// message DeleteShardsResponse { +// } + +// message ListShardsRequest { +// repeated ListShardsSubrequest subrequests = 1; +// } + +// message ListShardsSubrequest { +// string index_uid = 1; +// string source_id = 2; +// optional quickwit.ingest.ShardState shard_state = 3; +// } + +// message ListShardsResponse { +// repeated ListShardsSubresponse subresponses = 1; +// } + +// message ListShardsSubresponse { +// string index_uid = 1; +// string source_id = 2; +// repeated quickwit.ingest.Shard shards = 3; +// } diff --git a/quickwit/quickwit-proto/protos/quickwit/router.proto b/quickwit/quickwit-proto/protos/quickwit/router.proto new file mode 100644 index 00000000000..7364ba06ebb --- /dev/null +++ b/quickwit/quickwit-proto/protos/quickwit/router.proto @@ -0,0 +1,61 @@ +// 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 . + + +syntax = "proto3"; + +package quickwit.ingest.router; + +import "quickwit/ingest.proto"; + +service IngestRouterService { + /// Ingests batches of documents for one or multiple indexes. + /// TODO: Describe error cases and how to handle them. + rpc Ingest(IngestRequestV2) returns (IngestResponseV2); +} + +message IngestRequestV2 { + repeated IngestSubrequest subrequests = 1; + quickwit.ingest.CommitTypeV2 commit_type = 2; +} + +message IngestSubrequest { + string index_id = 1; + string source_id = 2; + quickwit.ingest.DocBatchV2 doc_batch = 3; +} + +message IngestResponseV2 { + repeated IngestSuccess successes = 1; + repeated IngestFailure failures = 2; +} + +message IngestSuccess { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + uint64 replication_position = 4; +} + +message IngestFailure { + string index_uid = 1; + string source_id = 2; + uint64 shard_id = 3; + // quickwit.ingest.DocBatchV2 doc_batch = 4; +} 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 3dfa43efb20..90c5f3705e0 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -6,6 +6,68 @@ pub struct NotifyIndexChangeRequest {} #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct NotifyIndexChangeResponse {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct GetOpenShardsRequest { + #[prost(message, repeated, tag = "1")] + pub subrequests: ::prost::alloc::vec::Vec, + #[prost(string, repeated, tag = "2")] + pub unavailable_ingesters: ::prost::alloc::vec::Vec<::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 GetOpenShardsSubrequest { + #[prost(string, tag = "1")] + pub index_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::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 GetOpenShardsResponse { + #[prost(message, repeated, tag = "1")] + pub subresponses: ::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 GetOpenShardsSubresponse { + #[prost(string, tag = "1")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "3")] + pub open_shards: ::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 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 {} /// BEGIN quickwit-codegen use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] @@ -14,7 +76,15 @@ pub trait ControlPlaneService: std::fmt::Debug + dyn_clone::DynClone + Send + Sy async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, - ) -> crate::control_plane::Result; + ) -> crate::control_plane::ControlPlaneResult; + async fn get_open_shards( + &mut self, + request: GetOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult; + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult; } dyn_clone::clone_trait_object!(ControlPlaneService); #[cfg(any(test, feature = "testsuite"))] @@ -78,9 +148,21 @@ impl ControlPlaneService for ControlPlaneServiceClient { async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, - ) -> crate::control_plane::Result { + ) -> crate::control_plane::ControlPlaneResult { self.inner.notify_index_change(request).await } + async fn get_open_shards( + &mut self, + request: GetOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.get_open_shards(request).await + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.close_shards(request).await + } } #[cfg(any(test, feature = "testsuite"))] pub mod mock { @@ -94,9 +176,21 @@ pub mod mock { async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, - ) -> crate::control_plane::Result { + ) -> crate::control_plane::ControlPlaneResult { self.inner.lock().await.notify_index_change(request).await } + async fn get_open_shards( + &mut self, + request: GetOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.lock().await.get_open_shards(request).await + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.lock().await.close_shards(request).await + } } impl From for ControlPlaneServiceClient { fn from(mock: MockControlPlaneService) -> Self { @@ -126,6 +220,38 @@ impl tower::Service for Box { Box::pin(fut) } } +impl tower::Service for Box { + type Response = GetOpenShardsResponse; + 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: GetOpenShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.get_open_shards(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = CloseShardsResponse; + 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: CloseShardsRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.close_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 { @@ -134,11 +260,23 @@ struct ControlPlaneServiceTowerBlock { NotifyIndexChangeResponse, crate::control_plane::ControlPlaneError, >, + get_open_shards_svc: quickwit_common::tower::BoxService< + GetOpenShardsRequest, + GetOpenShardsResponse, + crate::control_plane::ControlPlaneError, + >, + close_shards_svc: quickwit_common::tower::BoxService< + CloseShardsRequest, + CloseShardsResponse, + crate::control_plane::ControlPlaneError, + >, } impl Clone for ControlPlaneServiceTowerBlock { fn clone(&self) -> Self { Self { notify_index_change_svc: self.notify_index_change_svc.clone(), + get_open_shards_svc: self.get_open_shards_svc.clone(), + close_shards_svc: self.close_shards_svc.clone(), } } } @@ -147,9 +285,21 @@ impl ControlPlaneService for ControlPlaneServiceTowerBlock { async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, - ) -> crate::control_plane::Result { + ) -> crate::control_plane::ControlPlaneResult { self.notify_index_change_svc.ready().await?.call(request).await } + async fn get_open_shards( + &mut self, + request: GetOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.get_open_shards_svc.ready().await?.call(request).await + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.close_shards_svc.ready().await?.call(request).await + } } #[derive(Debug, Default)] pub struct ControlPlaneServiceTowerBlockBuilder { @@ -162,6 +312,24 @@ pub struct ControlPlaneServiceTowerBlockBuilder { crate::control_plane::ControlPlaneError, >, >, + #[allow(clippy::type_complexity)] + get_open_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + GetOpenShardsRequest, + GetOpenShardsResponse, + crate::control_plane::ControlPlaneError, + >, + >, + #[allow(clippy::type_complexity)] + close_shards_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + CloseShardsRequest, + CloseShardsResponse, + crate::control_plane::ControlPlaneError, + >, + >, } impl ControlPlaneServiceTowerBlockBuilder { pub fn shared_layer(mut self, layer: L) -> Self @@ -173,11 +341,28 @@ impl ControlPlaneServiceTowerBlockBuilder { Error = crate::control_plane::ControlPlaneError, > + Clone + Send + Sync + 'static, >::Future: Send + 'static, + L::Service: tower::Service< + GetOpenShardsRequest, + Response = GetOpenShardsResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + CloseShardsRequest, + Response = CloseShardsResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, { self .notify_index_change_layer = Some( - quickwit_common::tower::BoxLayer::new(layer), + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .get_open_shards_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), ); + self.close_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); self } pub fn notify_index_change_layer(mut self, layer: L) -> Self @@ -196,6 +381,32 @@ impl ControlPlaneServiceTowerBlockBuilder { ); self } + pub fn get_open_shards_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + GetOpenShardsRequest, + Response = GetOpenShardsResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.get_open_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::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.close_shards_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn build(self, instance: T) -> ControlPlaneServiceClient where T: ControlPlaneService, @@ -240,8 +451,20 @@ impl ControlPlaneServiceTowerBlockBuilder { } else { quickwit_common::tower::BoxService::new(boxed_instance.clone()) }; + let get_open_shards_svc = if let Some(layer) = self.get_open_shards_layer { + layer.layer(boxed_instance.clone()) + } 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 tower_block = ControlPlaneServiceTowerBlock { notify_index_change_svc, + get_open_shards_svc, + close_shards_svc, }; ControlPlaneServiceClient::new(tower_block) } @@ -319,19 +542,49 @@ where ControlPlaneServiceMailbox< A, >: tower::Service< - NotifyIndexChangeRequest, - Response = NotifyIndexChangeResponse, - Error = crate::control_plane::ControlPlaneError, - Future = BoxFuture< - NotifyIndexChangeResponse, - crate::control_plane::ControlPlaneError, + NotifyIndexChangeRequest, + Response = NotifyIndexChangeResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + NotifyIndexChangeResponse, + crate::control_plane::ControlPlaneError, + >, + > + + tower::Service< + GetOpenShardsRequest, + Response = GetOpenShardsResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + GetOpenShardsResponse, + crate::control_plane::ControlPlaneError, + >, + > + + tower::Service< + CloseShardsRequest, + Response = CloseShardsResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + CloseShardsResponse, + crate::control_plane::ControlPlaneError, + >, >, - >, { async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, - ) -> crate::control_plane::Result { + ) -> crate::control_plane::ControlPlaneResult { + self.call(request).await + } + async fn get_open_shards( + &mut self, + request: GetOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.call(request).await + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { self.call(request).await } } @@ -360,13 +613,33 @@ where async fn notify_index_change( &mut self, request: NotifyIndexChangeRequest, - ) -> crate::control_plane::Result { + ) -> crate::control_plane::ControlPlaneResult { self.inner .notify_index_change(request) .await .map(|response| response.into_inner()) .map_err(|error| error.into()) } + async fn get_open_shards( + &mut self, + request: GetOpenShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner + .get_open_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn close_shards( + &mut self, + request: CloseShardsRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner + .close_shards(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } } #[derive(Debug)] pub struct ControlPlaneServiceGrpcServerAdapter { @@ -394,6 +667,28 @@ for ControlPlaneServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(|error| error.into()) } + async fn get_open_shards( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .get_open_shards(request.into_inner()) + .await + .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()) + } } /// Generated client implementations. pub mod control_plane_service_grpc_client { @@ -506,14 +801,76 @@ pub mod control_plane_service_grpc_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.control_plane.ControlPlaneService/notifyIndexChange", + "/quickwit.control_plane.ControlPlaneService/NotifyIndexChange", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.control_plane.ControlPlaneService", + "NotifyIndexChange", + ), + ); + self.inner.unary(req, path, codec).await + } + /// / 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. + pub async fn get_open_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/GetOpenShards", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.control_plane.ControlPlaneService", + "GetOpenShards", + ), + ); + 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.control_plane.ControlPlaneService/CloseShards", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.control_plane.ControlPlaneService", - "notifyIndexChange", + "CloseShards", ), ); self.inner.unary(req, path, codec).await @@ -540,6 +897,22 @@ pub mod control_plane_service_grpc_server { tonic::Response, tonic::Status, >; + /// / 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. + async fn get_open_shards( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + async fn close_shards( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; } #[derive(Debug)] pub struct ControlPlaneServiceGrpcServer { @@ -621,13 +994,13 @@ pub mod control_plane_service_grpc_server { fn call(&mut self, req: http::Request) -> Self::Future { let inner = self.inner.clone(); match req.uri().path() { - "/quickwit.control_plane.ControlPlaneService/notifyIndexChange" => { + "/quickwit.control_plane.ControlPlaneService/NotifyIndexChange" => { #[allow(non_camel_case_types)] - struct notifyIndexChangeSvc(pub Arc); + struct NotifyIndexChangeSvc(pub Arc); impl< T: ControlPlaneServiceGrpc, > tonic::server::UnaryService - for notifyIndexChangeSvc { + for NotifyIndexChangeSvc { type Response = super::NotifyIndexChangeResponse; type Future = BoxFuture< tonic::Response, @@ -651,7 +1024,99 @@ pub mod control_plane_service_grpc_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = notifyIndexChangeSvc(inner); + let method = NotifyIndexChangeSvc(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/GetOpenShards" => { + #[allow(non_camel_case_types)] + struct GetOpenShardsSvc(pub Arc); + impl< + T: ControlPlaneServiceGrpc, + > tonic::server::UnaryService + for GetOpenShardsSvc { + type Response = super::GetOpenShardsResponse; + 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).get_open_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 = GetOpenShardsSvc(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/CloseShards" => { + #[allow(non_camel_case_types)] + struct CloseShardsSvc(pub Arc); + impl< + T: ControlPlaneServiceGrpc, + > 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( diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs index ddc2e8a94f3..81e964bd55b 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs @@ -5,22 +5,25 @@ pub struct ApplyIndexingPlanRequest { #[prost(message, repeated, tag = "1")] pub indexing_tasks: ::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 ApplyIndexingPlanResponse {} #[derive(Eq, Hash)] #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct IndexingTask { - /// / Index UID of the task. + /// / The tasks's index UID. #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, - /// / Source ID of the task. + /// / The task's source ID. + /// + /// / The shards assigned to the indexer. + /// repeated uint64 shard_ids = 3; #[prost(string, tag = "2")] pub source_id: ::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 ApplyIndexingPlanResponse {} /// BEGIN quickwit-codegen use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] @@ -29,7 +32,7 @@ pub trait IndexingService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + async fn apply_indexing_plan( &mut self, request: ApplyIndexingPlanRequest, - ) -> crate::indexing::Result; + ) -> crate::indexing::IndexingResult; } dyn_clone::clone_trait_object!(IndexingService); #[cfg(any(test, feature = "testsuite"))] @@ -91,7 +94,7 @@ impl IndexingService for IndexingServiceClient { async fn apply_indexing_plan( &mut self, request: ApplyIndexingPlanRequest, - ) -> crate::indexing::Result { + ) -> crate::indexing::IndexingResult { self.inner.apply_indexing_plan(request).await } } @@ -107,7 +110,7 @@ pub mod mock { async fn apply_indexing_plan( &mut self, request: ApplyIndexingPlanRequest, - ) -> crate::indexing::Result { + ) -> crate::indexing::IndexingResult { self.inner.lock().await.apply_indexing_plan(request).await } } @@ -160,7 +163,7 @@ impl IndexingService for IndexingServiceTowerBlock { async fn apply_indexing_plan( &mut self, request: ApplyIndexingPlanRequest, - ) -> crate::indexing::Result { + ) -> crate::indexing::IndexingResult { self.apply_indexing_plan_svc.ready().await?.call(request).await } } @@ -341,7 +344,7 @@ where async fn apply_indexing_plan( &mut self, request: ApplyIndexingPlanRequest, - ) -> crate::indexing::Result { + ) -> crate::indexing::IndexingResult { self.call(request).await } } @@ -370,7 +373,7 @@ where async fn apply_indexing_plan( &mut self, request: ApplyIndexingPlanRequest, - ) -> crate::indexing::Result { + ) -> crate::indexing::IndexingResult { self.inner .apply_indexing_plan(request) .await @@ -509,14 +512,14 @@ pub mod indexing_service_grpc_client { })?; let codec = tonic::codec::ProstCodec::default(); let path = http::uri::PathAndQuery::from_static( - "/quickwit.indexing.IndexingService/applyIndexingPlan", + "/quickwit.indexing.IndexingService/ApplyIndexingPlan", ); let mut req = request.into_request(); req.extensions_mut() .insert( GrpcMethod::new( "quickwit.indexing.IndexingService", - "applyIndexingPlan", + "ApplyIndexingPlan", ), ); self.inner.unary(req, path, codec).await @@ -618,13 +621,13 @@ pub mod indexing_service_grpc_server { fn call(&mut self, req: http::Request) -> Self::Future { let inner = self.inner.clone(); match req.uri().path() { - "/quickwit.indexing.IndexingService/applyIndexingPlan" => { + "/quickwit.indexing.IndexingService/ApplyIndexingPlan" => { #[allow(non_camel_case_types)] - struct applyIndexingPlanSvc(pub Arc); + struct ApplyIndexingPlanSvc(pub Arc); impl< T: IndexingServiceGrpc, > tonic::server::UnaryService - for applyIndexingPlanSvc { + for ApplyIndexingPlanSvc { type Response = super::ApplyIndexingPlanResponse; type Future = BoxFuture< tonic::Response, @@ -648,7 +651,7 @@ pub mod indexing_service_grpc_server { let inner = self.inner.clone(); let fut = async move { let inner = inner.0; - let method = applyIndexingPlanSvc(inner); + let method = ApplyIndexingPlanSvc(inner); let codec = tonic::codec::ProstCodec::default(); let mut grpc = tonic::server::Grpc::new(codec) .apply_compression_config( diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs new file mode 100644 index 00000000000..6f4a2f8f35f --- /dev/null +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -0,0 +1,1773 @@ +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PersistRequest { + #[prost(string, tag = "1")] + pub leader_id: ::prost::alloc::string::String, + #[prost(enumeration = "super::CommitTypeV2", tag = "3")] + pub commit_type: i32, + #[prost(message, repeated, tag = "4")] + 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 PersistSubrequest { + #[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, optional, tag = "4")] + pub follower_id: ::core::option::Option<::prost::alloc::string::String>, + #[prost(message, optional, tag = "5")] + pub doc_batch: ::core::option::Option, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PersistResponse { + #[prost(string, tag = "1")] + pub leader_id: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "2")] + pub successes: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + 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 PersistSuccess { + #[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(uint64, optional, tag = "4")] + 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 PersistFailure { + #[prost(string, tag = "1")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + /// ingest.DocBatchV2 doc_batch = 4; + /// ingest.IngestError error = 5; + #[prost(uint64, tag = "3")] + pub shard_id: u64, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct SynReplicationMessage { + #[prost(oneof = "syn_replication_message::Message", tags = "1, 2, 3")] + pub message: ::core::option::Option, +} +/// Nested message and enum types in `SynReplicationMessage`. +pub mod syn_replication_message { + #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] + #[serde(rename_all = "snake_case")] + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Message { + #[prost(message, tag = "1")] + OpenRequest(super::OpenReplicationStreamRequest), + #[prost(message, tag = "2")] + ReplicateRequest(super::ReplicateRequest), + #[prost(message, tag = "3")] + TruncateRequest(super::TruncateRequest), + } +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct AckReplicationMessage { + #[prost(oneof = "ack_replication_message::Message", tags = "1, 3, 4")] + pub message: ::core::option::Option, +} +/// Nested message and enum types in `AckReplicationMessage`. +pub mod ack_replication_message { + #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] + #[serde(rename_all = "snake_case")] + #[allow(clippy::derive_partial_eq_without_eq)] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Message { + #[prost(message, tag = "1")] + OpenResponse(super::OpenReplicationStreamResponse), + #[prost(message, tag = "3")] + ReplicateResponse(super::ReplicateResponse), + #[prost(message, tag = "4")] + TruncateResponse(super::TruncateResponse), + } +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct OpenReplicationStreamRequest { + #[prost(string, tag = "1")] + pub leader_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub follower_id: ::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 OpenReplicationStreamResponse {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReplicateRequest { + #[prost(string, tag = "1")] + pub leader_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub follower_id: ::prost::alloc::string::String, + #[prost(enumeration = "super::CommitTypeV2", tag = "3")] + pub commit_type: i32, + #[prost(message, repeated, tag = "4")] + 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 ReplicateSubrequest { + #[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(uint64, optional, tag = "4")] + pub from_position_exclusive: ::core::option::Option, + #[prost(message, optional, tag = "6")] + pub doc_batch: ::core::option::Option, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct ReplicateResponse { + #[prost(string, tag = "1")] + pub follower_id: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "2")] + pub successes: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "3")] + 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 ReplicateSuccess { + #[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(uint64, optional, tag = "4")] + pub replica_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 ReplicateFailure { + #[prost(string, tag = "1")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + /// ingest.DocBatchV2 doc_batch = 4; + /// ingest.IngestError error = 5; + #[prost(uint64, tag = "3")] + pub shard_id: u64, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TruncateRequest { + #[prost(string, tag = "1")] + pub leader_id: ::prost::alloc::string::String, + #[prost(message, repeated, tag = "2")] + 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 TruncateSubrequest { + #[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(uint64, tag = "4")] + pub to_position_inclusive: u64, +} +/// TODO +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct TruncateResponse {} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct OpenFetchStreamRequest { + #[prost(string, tag = "1")] + pub client_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "3")] + pub source_id: ::prost::alloc::string::String, + #[prost(uint64, tag = "4")] + pub shard_id: u64, + #[prost(uint64, optional, tag = "5")] + pub from_position_exclusive: ::core::option::Option, + #[prost(uint64, optional, tag = "6")] + pub to_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 FetchResponseV2 { + #[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(uint64, tag = "4")] + pub from_position_inclusive: u64, + #[prost(message, optional, tag = "5")] + pub doc_batch: ::core::option::Option, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct PingRequest { + #[prost(string, tag = "1")] + pub leader_id: ::prost::alloc::string::String, + #[prost(string, optional, tag = "2")] + pub follower_id: ::core::option::Option<::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 PingResponse {} +/// BEGIN quickwit-codegen +use tower::{Layer, Service, ServiceExt}; +pub type IngesterServiceStream = quickwit_common::ServiceStream< + crate::ingest::IngestV2Result, +>; +#[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] +#[async_trait::async_trait] +pub trait IngesterService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { + async fn persist( + &mut self, + request: PersistRequest, + ) -> crate::ingest::IngestV2Result; + async fn open_replication_stream( + &mut self, + request: quickwit_common::ServiceStream, + ) -> crate::ingest::IngestV2Result>; + async fn open_fetch_stream( + &mut self, + request: OpenFetchStreamRequest, + ) -> crate::ingest::IngestV2Result>; + async fn ping( + &mut self, + request: PingRequest, + ) -> crate::ingest::IngestV2Result; + async fn truncate( + &mut self, + request: TruncateRequest, + ) -> crate::ingest::IngestV2Result; +} +dyn_clone::clone_trait_object!(IngesterService); +#[cfg(any(test, feature = "testsuite"))] +impl Clone for MockIngesterService { + fn clone(&self) -> Self { + MockIngesterService::new() + } +} +#[derive(Debug, Clone)] +pub struct IngesterServiceClient { + inner: Box, +} +impl IngesterServiceClient { + pub fn new(instance: T) -> Self + where + T: IngesterService, + { + Self { inner: Box::new(instance) } + } + pub fn from_channel(channel: C) -> Self + where + C: tower::Service< + http::Request, + Response = http::Response, + Error = quickwit_common::tower::BoxError, + > + std::fmt::Debug + Clone + Send + Sync + 'static, + , + >>::Future: std::future::Future< + Output = Result< + http::Response, + quickwit_common::tower::BoxError, + >, + > + Send + 'static, + { + IngesterServiceClient::new( + IngesterServiceGrpcClientAdapter::new( + ingester_service_grpc_client::IngesterServiceGrpcClient::new(channel), + ), + ) + } + pub fn from_mailbox(mailbox: quickwit_actors::Mailbox) -> Self + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + IngesterServiceMailbox: IngesterService, + { + IngesterServiceClient::new(IngesterServiceMailbox::new(mailbox)) + } + pub fn tower() -> IngesterServiceTowerBlockBuilder { + IngesterServiceTowerBlockBuilder::default() + } + #[cfg(any(test, feature = "testsuite"))] + pub fn mock() -> MockIngesterService { + MockIngesterService::new() + } +} +#[async_trait::async_trait] +impl IngesterService for IngesterServiceClient { + async fn persist( + &mut self, + request: PersistRequest, + ) -> crate::ingest::IngestV2Result { + self.inner.persist(request).await + } + async fn open_replication_stream( + &mut self, + request: quickwit_common::ServiceStream, + ) -> crate::ingest::IngestV2Result> { + self.inner.open_replication_stream(request).await + } + async fn open_fetch_stream( + &mut self, + request: OpenFetchStreamRequest, + ) -> crate::ingest::IngestV2Result> { + self.inner.open_fetch_stream(request).await + } + async fn ping( + &mut self, + request: PingRequest, + ) -> crate::ingest::IngestV2Result { + self.inner.ping(request).await + } + async fn truncate( + &mut self, + request: TruncateRequest, + ) -> crate::ingest::IngestV2Result { + self.inner.truncate(request).await + } +} +#[cfg(any(test, feature = "testsuite"))] +pub mod mock { + use super::*; + #[derive(Debug, Clone)] + struct MockIngesterServiceWrapper { + inner: std::sync::Arc>, + } + #[async_trait::async_trait] + impl IngesterService for MockIngesterServiceWrapper { + async fn persist( + &mut self, + request: PersistRequest, + ) -> crate::ingest::IngestV2Result { + self.inner.lock().await.persist(request).await + } + async fn open_replication_stream( + &mut self, + request: quickwit_common::ServiceStream, + ) -> crate::ingest::IngestV2Result< + IngesterServiceStream, + > { + self.inner.lock().await.open_replication_stream(request).await + } + async fn open_fetch_stream( + &mut self, + request: OpenFetchStreamRequest, + ) -> crate::ingest::IngestV2Result> { + self.inner.lock().await.open_fetch_stream(request).await + } + async fn ping( + &mut self, + request: PingRequest, + ) -> crate::ingest::IngestV2Result { + self.inner.lock().await.ping(request).await + } + async fn truncate( + &mut self, + request: TruncateRequest, + ) -> crate::ingest::IngestV2Result { + self.inner.lock().await.truncate(request).await + } + } + impl From for IngesterServiceClient { + fn from(mock: MockIngesterService) -> Self { + let mock_wrapper = MockIngesterServiceWrapper { + inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + }; + IngesterServiceClient::new(mock_wrapper) + } + } +} +pub type BoxFuture = std::pin::Pin< + Box> + Send + 'static>, +>; +impl tower::Service for Box { + type Response = PersistResponse; + type Error = crate::ingest::IngestV2Error; + 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: PersistRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.persist(request).await }; + Box::pin(fut) + } +} +impl tower::Service> +for Box { + type Response = IngesterServiceStream; + type Error = crate::ingest::IngestV2Error; + 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: quickwit_common::ServiceStream, + ) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.open_replication_stream(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = IngesterServiceStream; + type Error = crate::ingest::IngestV2Error; + 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: OpenFetchStreamRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.open_fetch_stream(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = PingResponse; + type Error = crate::ingest::IngestV2Error; + 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: PingRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.ping(request).await }; + Box::pin(fut) + } +} +impl tower::Service for Box { + type Response = TruncateResponse; + type Error = crate::ingest::IngestV2Error; + 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: TruncateRequest) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.truncate(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 IngesterServiceTowerBlock { + persist_svc: quickwit_common::tower::BoxService< + PersistRequest, + PersistResponse, + crate::ingest::IngestV2Error, + >, + open_replication_stream_svc: quickwit_common::tower::BoxService< + quickwit_common::ServiceStream, + IngesterServiceStream, + crate::ingest::IngestV2Error, + >, + open_fetch_stream_svc: quickwit_common::tower::BoxService< + OpenFetchStreamRequest, + IngesterServiceStream, + crate::ingest::IngestV2Error, + >, + ping_svc: quickwit_common::tower::BoxService< + PingRequest, + PingResponse, + crate::ingest::IngestV2Error, + >, + truncate_svc: quickwit_common::tower::BoxService< + TruncateRequest, + TruncateResponse, + crate::ingest::IngestV2Error, + >, +} +impl Clone for IngesterServiceTowerBlock { + fn clone(&self) -> Self { + Self { + persist_svc: self.persist_svc.clone(), + open_replication_stream_svc: self.open_replication_stream_svc.clone(), + open_fetch_stream_svc: self.open_fetch_stream_svc.clone(), + ping_svc: self.ping_svc.clone(), + truncate_svc: self.truncate_svc.clone(), + } + } +} +#[async_trait::async_trait] +impl IngesterService for IngesterServiceTowerBlock { + async fn persist( + &mut self, + request: PersistRequest, + ) -> crate::ingest::IngestV2Result { + self.persist_svc.ready().await?.call(request).await + } + async fn open_replication_stream( + &mut self, + request: quickwit_common::ServiceStream, + ) -> crate::ingest::IngestV2Result> { + self.open_replication_stream_svc.ready().await?.call(request).await + } + async fn open_fetch_stream( + &mut self, + request: OpenFetchStreamRequest, + ) -> crate::ingest::IngestV2Result> { + self.open_fetch_stream_svc.ready().await?.call(request).await + } + async fn ping( + &mut self, + request: PingRequest, + ) -> crate::ingest::IngestV2Result { + self.ping_svc.ready().await?.call(request).await + } + async fn truncate( + &mut self, + request: TruncateRequest, + ) -> crate::ingest::IngestV2Result { + self.truncate_svc.ready().await?.call(request).await + } +} +#[derive(Debug, Default)] +pub struct IngesterServiceTowerBlockBuilder { + #[allow(clippy::type_complexity)] + persist_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + PersistRequest, + PersistResponse, + crate::ingest::IngestV2Error, + >, + >, + #[allow(clippy::type_complexity)] + open_replication_stream_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + quickwit_common::ServiceStream, + IngesterServiceStream, + crate::ingest::IngestV2Error, + >, + >, + #[allow(clippy::type_complexity)] + open_fetch_stream_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + OpenFetchStreamRequest, + IngesterServiceStream, + crate::ingest::IngestV2Error, + >, + >, + #[allow(clippy::type_complexity)] + ping_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + PingRequest, + PingResponse, + crate::ingest::IngestV2Error, + >, + >, + #[allow(clippy::type_complexity)] + truncate_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + TruncateRequest, + TruncateResponse, + crate::ingest::IngestV2Error, + >, + >, +} +impl IngesterServiceTowerBlockBuilder { + pub fn shared_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Clone + Send + Sync + 'static, + L::Service: tower::Service< + PersistRequest, + Response = PersistResponse, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + quickwit_common::ServiceStream, + Response = IngesterServiceStream, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + , + >>::Future: Send + 'static, + L::Service: tower::Service< + OpenFetchStreamRequest, + Response = IngesterServiceStream, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + PingRequest, + Response = PingResponse, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + L::Service: tower::Service< + TruncateRequest, + Response = TruncateResponse, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.persist_layer = Some(quickwit_common::tower::BoxLayer::new(layer.clone())); + self + .open_replication_stream_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self + .open_fetch_stream_layer = Some( + quickwit_common::tower::BoxLayer::new(layer.clone()), + ); + self.ping_layer = Some(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.truncate_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn persist_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + PersistRequest, + Response = PersistResponse, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.persist_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn open_replication_stream_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + quickwit_common::ServiceStream, + Response = IngesterServiceStream, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + , + >>::Future: Send + 'static, + { + self + .open_replication_stream_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn open_fetch_stream_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + OpenFetchStreamRequest, + Response = IngesterServiceStream, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self + .open_fetch_stream_layer = Some( + quickwit_common::tower::BoxLayer::new(layer), + ); + self + } + pub fn ping_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + PingRequest, + Response = PingResponse, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.ping_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn truncate_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + TruncateRequest, + Response = TruncateResponse, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.truncate_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn build(self, instance: T) -> IngesterServiceClient + where + T: IngesterService, + { + self.build_from_boxed(Box::new(instance)) + } + pub fn build_from_channel(self, channel: C) -> IngesterServiceClient + where + C: tower::Service< + http::Request, + Response = http::Response, + Error = quickwit_common::tower::BoxError, + > + std::fmt::Debug + Clone + Send + Sync + 'static, + , + >>::Future: std::future::Future< + Output = Result< + http::Response, + quickwit_common::tower::BoxError, + >, + > + Send + 'static, + { + self.build_from_boxed(Box::new(IngesterServiceClient::from_channel(channel))) + } + pub fn build_from_mailbox( + self, + mailbox: quickwit_actors::Mailbox, + ) -> IngesterServiceClient + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + IngesterServiceMailbox: IngesterService, + { + self.build_from_boxed(Box::new(IngesterServiceClient::from_mailbox(mailbox))) + } + fn build_from_boxed( + self, + boxed_instance: Box, + ) -> IngesterServiceClient { + let persist_svc = if let Some(layer) = self.persist_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let open_replication_stream_svc = if let Some(layer) + = self.open_replication_stream_layer + { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let open_fetch_stream_svc = if let Some(layer) = self.open_fetch_stream_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let ping_svc = if let Some(layer) = self.ping_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let truncate_svc = if let Some(layer) = self.truncate_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let tower_block = IngesterServiceTowerBlock { + persist_svc, + open_replication_stream_svc, + open_fetch_stream_svc, + ping_svc, + truncate_svc, + }; + IngesterServiceClient::new(tower_block) + } +} +#[derive(Debug, Clone)] +struct MailboxAdapter { + inner: quickwit_actors::Mailbox, + phantom: std::marker::PhantomData, +} +impl std::ops::Deref for MailboxAdapter +where + A: quickwit_actors::Actor, +{ + type Target = quickwit_actors::Mailbox; + fn deref(&self) -> &Self::Target { + &self.inner + } +} +#[derive(Debug)] +pub struct IngesterServiceMailbox { + inner: MailboxAdapter, +} +impl IngesterServiceMailbox { + pub fn new(instance: quickwit_actors::Mailbox) -> Self { + let inner = MailboxAdapter { + inner: instance, + phantom: std::marker::PhantomData, + }; + Self { inner } + } +} +impl Clone for IngesterServiceMailbox { + fn clone(&self) -> Self { + let inner = MailboxAdapter { + inner: self.inner.clone(), + phantom: std::marker::PhantomData, + }; + Self { inner } + } +} +impl tower::Service for IngesterServiceMailbox +where + A: quickwit_actors::Actor + + quickwit_actors::DeferableReplyHandler> + Send + + 'static, + M: std::fmt::Debug + Send + 'static, + T: Send + 'static, + E: std::fmt::Debug + Send + 'static, + crate::ingest::IngestV2Error: From>, +{ + type Response = T; + type Error = crate::ingest::IngestV2Error; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + //! This does not work with balance middlewares such as `tower::balance::pool::Pool` because + //! this always returns `Poll::Ready`. The fix is to acquire a permit from the + //! mailbox in `poll_ready` and consume it in `call`. + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, message: M) -> Self::Future { + let mailbox = self.inner.clone(); + let fut = async move { + mailbox.ask_for_res(message).await.map_err(|error| error.into()) + }; + Box::pin(fut) + } +} +#[async_trait::async_trait] +impl IngesterService for IngesterServiceMailbox +where + A: quickwit_actors::Actor + std::fmt::Debug, + IngesterServiceMailbox< + A, + >: tower::Service< + PersistRequest, + Response = PersistResponse, + Error = crate::ingest::IngestV2Error, + Future = BoxFuture, + > + + tower::Service< + quickwit_common::ServiceStream, + Response = IngesterServiceStream, + Error = crate::ingest::IngestV2Error, + Future = BoxFuture< + IngesterServiceStream, + crate::ingest::IngestV2Error, + >, + > + + tower::Service< + OpenFetchStreamRequest, + Response = IngesterServiceStream, + Error = crate::ingest::IngestV2Error, + Future = BoxFuture< + IngesterServiceStream, + crate::ingest::IngestV2Error, + >, + > + + tower::Service< + PingRequest, + Response = PingResponse, + Error = crate::ingest::IngestV2Error, + Future = BoxFuture, + > + + tower::Service< + TruncateRequest, + Response = TruncateResponse, + Error = crate::ingest::IngestV2Error, + Future = BoxFuture, + >, +{ + async fn persist( + &mut self, + request: PersistRequest, + ) -> crate::ingest::IngestV2Result { + self.call(request).await + } + async fn open_replication_stream( + &mut self, + request: quickwit_common::ServiceStream, + ) -> crate::ingest::IngestV2Result> { + self.call(request).await + } + async fn open_fetch_stream( + &mut self, + request: OpenFetchStreamRequest, + ) -> crate::ingest::IngestV2Result> { + self.call(request).await + } + async fn ping( + &mut self, + request: PingRequest, + ) -> crate::ingest::IngestV2Result { + self.call(request).await + } + async fn truncate( + &mut self, + request: TruncateRequest, + ) -> crate::ingest::IngestV2Result { + self.call(request).await + } +} +#[derive(Debug, Clone)] +pub struct IngesterServiceGrpcClientAdapter { + inner: T, +} +impl IngesterServiceGrpcClientAdapter { + pub fn new(instance: T) -> Self { + Self { inner: instance } + } +} +#[async_trait::async_trait] +impl IngesterService +for IngesterServiceGrpcClientAdapter< + ingester_service_grpc_client::IngesterServiceGrpcClient, +> +where + T: tonic::client::GrpcService + std::fmt::Debug + Clone + Send + + Sync + 'static, + T::ResponseBody: tonic::codegen::Body + Send + 'static, + ::Error: Into + + Send, + T::Future: Send, +{ + async fn persist( + &mut self, + request: PersistRequest, + ) -> crate::ingest::IngestV2Result { + self.inner + .persist(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn open_replication_stream( + &mut self, + request: quickwit_common::ServiceStream, + ) -> crate::ingest::IngestV2Result> { + self.inner + .open_replication_stream(request) + .await + .map(|response| { + let streaming: tonic::Streaming<_> = response.into_inner(); + let stream = quickwit_common::ServiceStream::from(streaming); + stream.map_err(|error| error.into()) + }) + .map_err(|error| error.into()) + } + async fn open_fetch_stream( + &mut self, + request: OpenFetchStreamRequest, + ) -> crate::ingest::IngestV2Result> { + self.inner + .open_fetch_stream(request) + .await + .map(|response| { + let streaming: tonic::Streaming<_> = response.into_inner(); + let stream = quickwit_common::ServiceStream::from(streaming); + stream.map_err(|error| error.into()) + }) + .map_err(|error| error.into()) + } + async fn ping( + &mut self, + request: PingRequest, + ) -> crate::ingest::IngestV2Result { + self.inner + .ping(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } + async fn truncate( + &mut self, + request: TruncateRequest, + ) -> crate::ingest::IngestV2Result { + self.inner + .truncate(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } +} +#[derive(Debug)] +pub struct IngesterServiceGrpcServerAdapter { + inner: Box, +} +impl IngesterServiceGrpcServerAdapter { + pub fn new(instance: T) -> Self + where + T: IngesterService, + { + Self { inner: Box::new(instance) } + } +} +#[async_trait::async_trait] +impl ingester_service_grpc_server::IngesterServiceGrpc +for IngesterServiceGrpcServerAdapter { + async fn persist( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .persist(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + type OpenReplicationStreamStream = quickwit_common::ServiceStream< + tonic::Result, + >; + async fn open_replication_stream( + &self, + request: tonic::Request>, + ) -> Result, tonic::Status> { + self.inner + .clone() + .open_replication_stream({ + let streaming: tonic::Streaming<_> = request.into_inner(); + quickwit_common::ServiceStream::from(streaming) + }) + .await + .map(|stream| tonic::Response::new(stream.map_err(|error| error.into()))) + .map_err(|error| error.into()) + } + type OpenFetchStreamStream = quickwit_common::ServiceStream< + tonic::Result, + >; + async fn open_fetch_stream( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .open_fetch_stream(request.into_inner()) + .await + .map(|stream| tonic::Response::new(stream.map_err(|error| error.into()))) + .map_err(|error| error.into()) + } + async fn ping( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .ping(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } + async fn truncate( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .truncate(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } +} +/// Generated client implementations. +pub mod ingester_service_grpc_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct IngesterServiceGrpcClient { + inner: tonic::client::Grpc, + } + impl IngesterServiceGrpcClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl IngesterServiceGrpcClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> IngesterServiceGrpcClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + IngesterServiceGrpcClient::new(InterceptedService::new(inner, interceptor)) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + /// / Persists batches of documents to primary shards owned by a leader. + pub async fn persist( + &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.ingest.ingester.IngesterService/Persist", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.ingest.ingester.IngesterService", + "Persist", + ), + ); + self.inner.unary(req, path, codec).await + } + /// / Opens a replication stream from a leader to a follower. + pub async fn open_replication_stream( + &mut self, + request: impl tonic::IntoStreamingRequest< + Message = super::SynReplicationMessage, + >, + ) -> 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.ingest.ingester.IngesterService/OpenReplicationStream", + ); + let mut req = request.into_streaming_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.ingest.ingester.IngesterService", + "OpenReplicationStream", + ), + ); + self.inner.streaming(req, path, codec).await + } + /// / Streams records from a leader or a follower. The client can optionally specify a range of positions to fetch. + pub async fn open_fetch_stream( + &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.ingest.ingester.IngesterService/OpenFetchStream", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.ingest.ingester.IngesterService", + "OpenFetchStream", + ), + ); + self.inner.server_streaming(req, path, codec).await + } + /// / Pings an ingester to check if it is ready to host shards and serve requests. + pub async fn ping( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, 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.ingest.ingester.IngesterService/Ping", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new("quickwit.ingest.ingester.IngesterService", "Ping"), + ); + self.inner.unary(req, path, codec).await + } + /// / Truncates the shards at the given positions. Callers should this RPC on leaders and they will forward to followers. + pub async fn truncate( + &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.ingest.ingester.IngesterService/Truncate", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.ingest.ingester.IngesterService", + "Truncate", + ), + ); + self.inner.unary(req, path, codec).await + } + } +} +/// Generated server implementations. +pub mod ingester_service_grpc_server { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + /// Generated trait containing gRPC methods that should be implemented for use with IngesterServiceGrpcServer. + #[async_trait] + pub trait IngesterServiceGrpc: Send + Sync + 'static { + /// / Persists batches of documents to primary shards owned by a leader. + async fn persist( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + /// Server streaming response type for the OpenReplicationStream method. + type OpenReplicationStreamStream: futures_core::Stream< + Item = std::result::Result, + > + + Send + + 'static; + /// / Opens a replication stream from a leader to a follower. + async fn open_replication_stream( + &self, + request: tonic::Request>, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + /// Server streaming response type for the OpenFetchStream method. + type OpenFetchStreamStream: futures_core::Stream< + Item = std::result::Result, + > + + Send + + 'static; + /// / Streams records from a leader or a follower. The client can optionally specify a range of positions to fetch. + async fn open_fetch_stream( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + /// / Pings an ingester to check if it is ready to host shards and serve requests. + async fn ping( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; + /// / Truncates the shards at the given positions. Callers should this RPC on leaders and they will forward to followers. + async fn truncate( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + } + #[derive(Debug)] + pub struct IngesterServiceGrpcServer { + inner: _Inner, + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, + } + struct _Inner(Arc); + impl IngesterServiceGrpcServer { + pub fn new(inner: T) -> Self { + Self::from_arc(Arc::new(inner)) + } + pub fn from_arc(inner: Arc) -> Self { + let inner = _Inner(inner); + Self { + inner, + accept_compression_encodings: Default::default(), + send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, + } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> InterceptedService + where + F: tonic::service::Interceptor, + { + InterceptedService::new(Self::new(inner), interceptor) + } + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } + } + impl tonic::codegen::Service> for IngesterServiceGrpcServer + where + T: IngesterServiceGrpc, + B: Body + Send + 'static, + B::Error: Into + Send + 'static, + { + type Response = http::Response; + type Error = std::convert::Infallible; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(Ok(())) + } + fn call(&mut self, req: http::Request) -> Self::Future { + let inner = self.inner.clone(); + match req.uri().path() { + "/quickwit.ingest.ingester.IngesterService/Persist" => { + #[allow(non_camel_case_types)] + struct PersistSvc(pub Arc); + impl< + T: IngesterServiceGrpc, + > tonic::server::UnaryService + for PersistSvc { + type Response = super::PersistResponse; + 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).persist(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 = PersistSvc(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.ingest.ingester.IngesterService/OpenReplicationStream" => { + #[allow(non_camel_case_types)] + struct OpenReplicationStreamSvc(pub Arc); + impl< + T: IngesterServiceGrpc, + > tonic::server::StreamingService + for OpenReplicationStreamSvc { + type Response = super::AckReplicationMessage; + type ResponseStream = T::OpenReplicationStreamStream; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request< + tonic::Streaming, + >, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + (*inner).open_replication_stream(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 = OpenReplicationStreamSvc(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.streaming(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + "/quickwit.ingest.ingester.IngesterService/OpenFetchStream" => { + #[allow(non_camel_case_types)] + struct OpenFetchStreamSvc(pub Arc); + impl< + T: IngesterServiceGrpc, + > tonic::server::ServerStreamingService< + super::OpenFetchStreamRequest, + > for OpenFetchStreamSvc { + type Response = super::FetchResponseV2; + type ResponseStream = T::OpenFetchStreamStream; + 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).open_fetch_stream(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 = OpenFetchStreamSvc(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.server_streaming(method, req).await; + Ok(res) + }; + Box::pin(fut) + } + "/quickwit.ingest.ingester.IngesterService/Ping" => { + #[allow(non_camel_case_types)] + struct PingSvc(pub Arc); + impl< + T: IngesterServiceGrpc, + > tonic::server::UnaryService for PingSvc { + type Response = super::PingResponse; + 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).ping(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 = PingSvc(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.ingest.ingester.IngesterService/Truncate" => { + #[allow(non_camel_case_types)] + struct TruncateSvc(pub Arc); + impl< + T: IngesterServiceGrpc, + > tonic::server::UnaryService + for TruncateSvc { + type Response = super::TruncateResponse; + 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).truncate(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 = TruncateSvc(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( + http::Response::builder() + .status(200) + .header("grpc-status", "12") + .header("content-type", "application/grpc") + .body(empty_body()) + .unwrap(), + ) + }) + } + } + } + } + impl Clone for IngesterServiceGrpcServer { + fn clone(&self) -> Self { + let inner = self.inner.clone(); + Self { + inner, + accept_compression_encodings: self.accept_compression_encodings, + send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, + } + } + } + impl Clone for _Inner { + fn clone(&self) -> Self { + Self(Arc::clone(&self.0)) + } + } + impl std::fmt::Debug for _Inner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{:?}", self.0) + } + } + impl tonic::server::NamedService + for IngesterServiceGrpcServer { + const NAME: &'static str = "quickwit.ingest.ingester.IngesterService"; + } +} diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs new file mode 100644 index 00000000000..471f38be825 --- /dev/null +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -0,0 +1,737 @@ +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct IngestRequestV2 { + #[prost(message, repeated, tag = "1")] + pub subrequests: ::prost::alloc::vec::Vec, + #[prost(enumeration = "super::CommitTypeV2", tag = "2")] + pub commit_type: i32, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct IngestSubrequest { + #[prost(string, tag = "1")] + pub index_id: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + #[prost(message, optional, tag = "3")] + pub doc_batch: ::core::option::Option, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct IngestResponseV2 { + #[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 IngestSuccess { + #[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(uint64, tag = "4")] + pub replication_position: u64, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct IngestFailure { + #[prost(string, tag = "1")] + pub index_uid: ::prost::alloc::string::String, + #[prost(string, tag = "2")] + pub source_id: ::prost::alloc::string::String, + /// quickwit.ingest.DocBatchV2 doc_batch = 4; + #[prost(uint64, tag = "3")] + pub shard_id: u64, +} +/// BEGIN quickwit-codegen +use tower::{Layer, Service, ServiceExt}; +#[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] +#[async_trait::async_trait] +pub trait IngestRouterService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { + async fn ingest( + &mut self, + request: IngestRequestV2, + ) -> crate::ingest::IngestV2Result; +} +dyn_clone::clone_trait_object!(IngestRouterService); +#[cfg(any(test, feature = "testsuite"))] +impl Clone for MockIngestRouterService { + fn clone(&self) -> Self { + MockIngestRouterService::new() + } +} +#[derive(Debug, Clone)] +pub struct IngestRouterServiceClient { + inner: Box, +} +impl IngestRouterServiceClient { + pub fn new(instance: T) -> Self + where + T: IngestRouterService, + { + Self { inner: Box::new(instance) } + } + pub fn from_channel(channel: C) -> Self + where + C: tower::Service< + http::Request, + Response = http::Response, + Error = quickwit_common::tower::BoxError, + > + std::fmt::Debug + Clone + Send + Sync + 'static, + , + >>::Future: std::future::Future< + Output = Result< + http::Response, + quickwit_common::tower::BoxError, + >, + > + Send + 'static, + { + IngestRouterServiceClient::new( + IngestRouterServiceGrpcClientAdapter::new( + ingest_router_service_grpc_client::IngestRouterServiceGrpcClient::new( + channel, + ), + ), + ) + } + pub fn from_mailbox(mailbox: quickwit_actors::Mailbox) -> Self + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + IngestRouterServiceMailbox: IngestRouterService, + { + IngestRouterServiceClient::new(IngestRouterServiceMailbox::new(mailbox)) + } + pub fn tower() -> IngestRouterServiceTowerBlockBuilder { + IngestRouterServiceTowerBlockBuilder::default() + } + #[cfg(any(test, feature = "testsuite"))] + pub fn mock() -> MockIngestRouterService { + MockIngestRouterService::new() + } +} +#[async_trait::async_trait] +impl IngestRouterService for IngestRouterServiceClient { + async fn ingest( + &mut self, + request: IngestRequestV2, + ) -> crate::ingest::IngestV2Result { + self.inner.ingest(request).await + } +} +#[cfg(any(test, feature = "testsuite"))] +pub mod mock { + use super::*; + #[derive(Debug, Clone)] + struct MockIngestRouterServiceWrapper { + inner: std::sync::Arc>, + } + #[async_trait::async_trait] + impl IngestRouterService for MockIngestRouterServiceWrapper { + async fn ingest( + &mut self, + request: IngestRequestV2, + ) -> crate::ingest::IngestV2Result { + self.inner.lock().await.ingest(request).await + } + } + impl From for IngestRouterServiceClient { + fn from(mock: MockIngestRouterService) -> Self { + let mock_wrapper = MockIngestRouterServiceWrapper { + inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + }; + IngestRouterServiceClient::new(mock_wrapper) + } + } +} +pub type BoxFuture = std::pin::Pin< + Box> + Send + 'static>, +>; +impl tower::Service for Box { + type Response = IngestResponseV2; + type Error = crate::ingest::IngestV2Error; + 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: IngestRequestV2) -> Self::Future { + let mut svc = self.clone(); + let fut = async move { svc.ingest(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 IngestRouterServiceTowerBlock { + ingest_svc: quickwit_common::tower::BoxService< + IngestRequestV2, + IngestResponseV2, + crate::ingest::IngestV2Error, + >, +} +impl Clone for IngestRouterServiceTowerBlock { + fn clone(&self) -> Self { + Self { + ingest_svc: self.ingest_svc.clone(), + } + } +} +#[async_trait::async_trait] +impl IngestRouterService for IngestRouterServiceTowerBlock { + async fn ingest( + &mut self, + request: IngestRequestV2, + ) -> crate::ingest::IngestV2Result { + self.ingest_svc.ready().await?.call(request).await + } +} +#[derive(Debug, Default)] +pub struct IngestRouterServiceTowerBlockBuilder { + #[allow(clippy::type_complexity)] + ingest_layer: Option< + quickwit_common::tower::BoxLayer< + Box, + IngestRequestV2, + IngestResponseV2, + crate::ingest::IngestV2Error, + >, + >, +} +impl IngestRouterServiceTowerBlockBuilder { + pub fn shared_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Clone + Send + Sync + 'static, + L::Service: tower::Service< + IngestRequestV2, + Response = IngestResponseV2, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.ingest_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn ingest_layer(mut self, layer: L) -> Self + where + L: tower::Layer> + Send + Sync + 'static, + L::Service: tower::Service< + IngestRequestV2, + Response = IngestResponseV2, + Error = crate::ingest::IngestV2Error, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.ingest_layer = Some(quickwit_common::tower::BoxLayer::new(layer)); + self + } + pub fn build(self, instance: T) -> IngestRouterServiceClient + where + T: IngestRouterService, + { + self.build_from_boxed(Box::new(instance)) + } + pub fn build_from_channel(self, channel: C) -> IngestRouterServiceClient + where + C: tower::Service< + http::Request, + Response = http::Response, + Error = quickwit_common::tower::BoxError, + > + std::fmt::Debug + Clone + Send + Sync + 'static, + , + >>::Future: std::future::Future< + Output = Result< + http::Response, + quickwit_common::tower::BoxError, + >, + > + Send + 'static, + { + self.build_from_boxed(Box::new(IngestRouterServiceClient::from_channel(channel))) + } + pub fn build_from_mailbox( + self, + mailbox: quickwit_actors::Mailbox, + ) -> IngestRouterServiceClient + where + A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, + IngestRouterServiceMailbox: IngestRouterService, + { + self.build_from_boxed(Box::new(IngestRouterServiceClient::from_mailbox(mailbox))) + } + fn build_from_boxed( + self, + boxed_instance: Box, + ) -> IngestRouterServiceClient { + let ingest_svc = if let Some(layer) = self.ingest_layer { + layer.layer(boxed_instance.clone()) + } else { + quickwit_common::tower::BoxService::new(boxed_instance.clone()) + }; + let tower_block = IngestRouterServiceTowerBlock { + ingest_svc, + }; + IngestRouterServiceClient::new(tower_block) + } +} +#[derive(Debug, Clone)] +struct MailboxAdapter { + inner: quickwit_actors::Mailbox, + phantom: std::marker::PhantomData, +} +impl std::ops::Deref for MailboxAdapter +where + A: quickwit_actors::Actor, +{ + type Target = quickwit_actors::Mailbox; + fn deref(&self) -> &Self::Target { + &self.inner + } +} +#[derive(Debug)] +pub struct IngestRouterServiceMailbox { + inner: MailboxAdapter, +} +impl IngestRouterServiceMailbox { + pub fn new(instance: quickwit_actors::Mailbox) -> Self { + let inner = MailboxAdapter { + inner: instance, + phantom: std::marker::PhantomData, + }; + Self { inner } + } +} +impl Clone for IngestRouterServiceMailbox { + fn clone(&self) -> Self { + let inner = MailboxAdapter { + inner: self.inner.clone(), + phantom: std::marker::PhantomData, + }; + Self { inner } + } +} +impl tower::Service for IngestRouterServiceMailbox +where + A: quickwit_actors::Actor + + quickwit_actors::DeferableReplyHandler> + Send + + 'static, + M: std::fmt::Debug + Send + 'static, + T: Send + 'static, + E: std::fmt::Debug + Send + 'static, + crate::ingest::IngestV2Error: From>, +{ + type Response = T; + type Error = crate::ingest::IngestV2Error; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + //! This does not work with balance middlewares such as `tower::balance::pool::Pool` because + //! this always returns `Poll::Ready`. The fix is to acquire a permit from the + //! mailbox in `poll_ready` and consume it in `call`. + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, message: M) -> Self::Future { + let mailbox = self.inner.clone(); + let fut = async move { + mailbox.ask_for_res(message).await.map_err(|error| error.into()) + }; + Box::pin(fut) + } +} +#[async_trait::async_trait] +impl IngestRouterService for IngestRouterServiceMailbox +where + A: quickwit_actors::Actor + std::fmt::Debug, + IngestRouterServiceMailbox< + A, + >: tower::Service< + IngestRequestV2, + Response = IngestResponseV2, + Error = crate::ingest::IngestV2Error, + Future = BoxFuture, + >, +{ + async fn ingest( + &mut self, + request: IngestRequestV2, + ) -> crate::ingest::IngestV2Result { + self.call(request).await + } +} +#[derive(Debug, Clone)] +pub struct IngestRouterServiceGrpcClientAdapter { + inner: T, +} +impl IngestRouterServiceGrpcClientAdapter { + pub fn new(instance: T) -> Self { + Self { inner: instance } + } +} +#[async_trait::async_trait] +impl IngestRouterService +for IngestRouterServiceGrpcClientAdapter< + ingest_router_service_grpc_client::IngestRouterServiceGrpcClient, +> +where + T: tonic::client::GrpcService + std::fmt::Debug + Clone + Send + + Sync + 'static, + T::ResponseBody: tonic::codegen::Body + Send + 'static, + ::Error: Into + + Send, + T::Future: Send, +{ + async fn ingest( + &mut self, + request: IngestRequestV2, + ) -> crate::ingest::IngestV2Result { + self.inner + .ingest(request) + .await + .map(|response| response.into_inner()) + .map_err(|error| error.into()) + } +} +#[derive(Debug)] +pub struct IngestRouterServiceGrpcServerAdapter { + inner: Box, +} +impl IngestRouterServiceGrpcServerAdapter { + pub fn new(instance: T) -> Self + where + T: IngestRouterService, + { + Self { inner: Box::new(instance) } + } +} +#[async_trait::async_trait] +impl ingest_router_service_grpc_server::IngestRouterServiceGrpc +for IngestRouterServiceGrpcServerAdapter { + async fn ingest( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .clone() + .ingest(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(|error| error.into()) + } +} +/// Generated client implementations. +pub mod ingest_router_service_grpc_client { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + use tonic::codegen::http::Uri; + #[derive(Debug, Clone)] + pub struct IngestRouterServiceGrpcClient { + inner: tonic::client::Grpc, + } + impl IngestRouterServiceGrpcClient { + /// Attempt to create a new client by connecting to a given endpoint. + pub async fn connect(dst: D) -> Result + where + D: TryInto, + D::Error: Into, + { + let conn = tonic::transport::Endpoint::new(dst)?.connect().await?; + Ok(Self::new(conn)) + } + } + impl IngestRouterServiceGrpcClient + where + T: tonic::client::GrpcService, + T::Error: Into, + T::ResponseBody: Body + Send + 'static, + ::Error: Into + Send, + { + pub fn new(inner: T) -> Self { + let inner = tonic::client::Grpc::new(inner); + Self { inner } + } + pub fn with_origin(inner: T, origin: Uri) -> Self { + let inner = tonic::client::Grpc::with_origin(inner, origin); + Self { inner } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> IngestRouterServiceGrpcClient> + where + F: tonic::service::Interceptor, + T::ResponseBody: Default, + T: tonic::codegen::Service< + http::Request, + Response = http::Response< + >::ResponseBody, + >, + >, + , + >>::Error: Into + Send + Sync, + { + IngestRouterServiceGrpcClient::new( + InterceptedService::new(inner, interceptor), + ) + } + /// Compress requests with the given encoding. + /// + /// This requires the server to support it otherwise it might respond with an + /// error. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.send_compressed(encoding); + self + } + /// Enable decompressing responses. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.inner = self.inner.accept_compressed(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_decoding_message_size(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.inner = self.inner.max_encoding_message_size(limit); + self + } + /// / Ingests batches of documents for one or multiple indexes. + /// / TODO: Describe error cases and how to handle them. + pub async fn ingest( + &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.ingest.router.IngestRouterService/Ingest", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.ingest.router.IngestRouterService", + "Ingest", + ), + ); + self.inner.unary(req, path, codec).await + } + } +} +/// Generated server implementations. +pub mod ingest_router_service_grpc_server { + #![allow(unused_variables, dead_code, missing_docs, clippy::let_unit_value)] + use tonic::codegen::*; + /// Generated trait containing gRPC methods that should be implemented for use with IngestRouterServiceGrpcServer. + #[async_trait] + pub trait IngestRouterServiceGrpc: Send + Sync + 'static { + /// / Ingests batches of documents for one or multiple indexes. + /// / TODO: Describe error cases and how to handle them. + async fn ingest( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; + } + #[derive(Debug)] + pub struct IngestRouterServiceGrpcServer { + inner: _Inner, + accept_compression_encodings: EnabledCompressionEncodings, + send_compression_encodings: EnabledCompressionEncodings, + max_decoding_message_size: Option, + max_encoding_message_size: Option, + } + struct _Inner(Arc); + impl IngestRouterServiceGrpcServer { + pub fn new(inner: T) -> Self { + Self::from_arc(Arc::new(inner)) + } + pub fn from_arc(inner: Arc) -> Self { + let inner = _Inner(inner); + Self { + inner, + accept_compression_encodings: Default::default(), + send_compression_encodings: Default::default(), + max_decoding_message_size: None, + max_encoding_message_size: None, + } + } + pub fn with_interceptor( + inner: T, + interceptor: F, + ) -> InterceptedService + where + F: tonic::service::Interceptor, + { + InterceptedService::new(Self::new(inner), interceptor) + } + /// Enable decompressing requests with the given encoding. + #[must_use] + pub fn accept_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.accept_compression_encodings.enable(encoding); + self + } + /// Compress responses with the given encoding, if the client supports it. + #[must_use] + pub fn send_compressed(mut self, encoding: CompressionEncoding) -> Self { + self.send_compression_encodings.enable(encoding); + self + } + /// Limits the maximum size of a decoded message. + /// + /// Default: `4MB` + #[must_use] + pub fn max_decoding_message_size(mut self, limit: usize) -> Self { + self.max_decoding_message_size = Some(limit); + self + } + /// Limits the maximum size of an encoded message. + /// + /// Default: `usize::MAX` + #[must_use] + pub fn max_encoding_message_size(mut self, limit: usize) -> Self { + self.max_encoding_message_size = Some(limit); + self + } + } + impl tonic::codegen::Service> + for IngestRouterServiceGrpcServer + where + T: IngestRouterServiceGrpc, + B: Body + Send + 'static, + B::Error: Into + Send + 'static, + { + type Response = http::Response; + type Error = std::convert::Infallible; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut Context<'_>, + ) -> Poll> { + Poll::Ready(Ok(())) + } + fn call(&mut self, req: http::Request) -> Self::Future { + let inner = self.inner.clone(); + match req.uri().path() { + "/quickwit.ingest.router.IngestRouterService/Ingest" => { + #[allow(non_camel_case_types)] + struct IngestSvc(pub Arc); + impl< + T: IngestRouterServiceGrpc, + > tonic::server::UnaryService + for IngestSvc { + type Response = super::IngestResponseV2; + 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).ingest(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 = IngestSvc(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( + http::Response::builder() + .status(200) + .header("grpc-status", "12") + .header("content-type", "application/grpc") + .body(empty_body()) + .unwrap(), + ) + }) + } + } + } + } + impl Clone for IngestRouterServiceGrpcServer { + fn clone(&self) -> Self { + let inner = self.inner.clone(); + Self { + inner, + accept_compression_encodings: self.accept_compression_encodings, + send_compression_encodings: self.send_compression_encodings, + max_decoding_message_size: self.max_decoding_message_size, + max_encoding_message_size: self.max_encoding_message_size, + } + } + } + impl Clone for _Inner { + fn clone(&self) -> Self { + Self(Arc::clone(&self.0)) + } + } + impl std::fmt::Debug for _Inner { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{:?}", self.0) + } + } + impl tonic::server::NamedService + for IngestRouterServiceGrpcServer { + const NAME: &'static str = "quickwit.ingest.router.IngestRouterService"; + } +} diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs new file mode 100644 index 00000000000..43d82e36d5c --- /dev/null +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.rs @@ -0,0 +1,106 @@ +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct DocBatchV2 { + #[prost(bytes = "bytes", tag = "1")] + pub doc_buffer: ::prost::bytes::Bytes, + #[prost(uint32, repeated, tag = "2")] + pub doc_lengths: ::prost::alloc::vec::Vec, +} +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct Shard { + /// Immutable fields + #[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, + /// / The node ID of the ingester to which all the write requests for this shard should be sent to. + #[prost(string, tag = "4")] + pub leader_id: ::prost::alloc::string::String, + /// / The node ID of the ingester holding a copy of the data. + #[prost(string, optional, tag = "5")] + pub follower_id: ::core::option::Option<::prost::alloc::string::String>, + /// 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")] + 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")] + pub publish_position_inclusive: ::prost::alloc::string::String, + #[prost(string, optional, tag = "11")] + pub publish_token: ::core::option::Option<::prost::alloc::string::String>, +} +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum CommitTypeV2 { + Auto = 0, + Wait = 1, + Force = 2, +} +impl CommitTypeV2 { + /// 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 { + CommitTypeV2::Auto => "Auto", + CommitTypeV2::Wait => "Wait", + CommitTypeV2::Force => "Force", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "Auto" => Some(Self::Auto), + "Wait" => Some(Self::Wait), + "Force" => Some(Self::Force), + _ => None, + } + } +} +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +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 the its leader or follower to close it with its final replication position, after which write requests will be rejected. + Closing = 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, +} +impl ShardState { + /// 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 { + ShardState::Open => "OPEN", + ShardState::Closing => "CLOSING", + ShardState::Closed => "CLOSED", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "OPEN" => Some(Self::Open), + "CLOSING" => Some(Self::Closing), + "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 a5af0981ea6..d168d110a3b 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -89,9 +89,10 @@ pub struct PublishSplitsRequest { #[prost(string, tag = "1")] pub index_uid: ::prost::alloc::string::String, #[prost(string, repeated, tag = "2")] - pub split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + pub staged_split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, #[prost(string, repeated, tag = "3")] pub replaced_split_ids: ::prost::alloc::vec::Vec<::prost::alloc::string::String>, + /// optional string publish_token = 5; #[prost(string, optional, tag = "4")] pub index_checkpoint_delta_serialized_json: ::core::option::Option< ::prost::alloc::string::String, @@ -173,12 +174,12 @@ pub struct DeleteTask { pub delete_query: ::core::option::Option, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] -#[serde(default)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct DeleteQuery { /// Index ID. #[prost(string, tag = "1")] + #[serde(alias = "index_id")] pub index_uid: ::prost::alloc::string::String, /// If set, restrict search to documents with a `timestamp >= start_timestamp`. #[prost(int64, optional, tag = "2")] @@ -191,6 +192,7 @@ pub struct DeleteQuery { /// Query text. The query language is that of tantivy. /// Query AST serialized in JSON #[prost(string, tag = "6")] + #[serde(alias = "query")] pub query_ast: ::prost::alloc::string::String, } #[derive(Serialize, Deserialize, utoipa::ToSchema)] diff --git a/quickwit/quickwit-proto/src/control_plane/mod.rs b/quickwit/quickwit-proto/src/control_plane/mod.rs index 602ed62c6b6..fdebdd1b9c2 100644 --- a/quickwit/quickwit-proto/src/control_plane/mod.rs +++ b/quickwit/quickwit-proto/src/control_plane/mod.rs @@ -20,12 +20,9 @@ use quickwit_actors::AskError; use thiserror; -#[path = "../codegen/quickwit/quickwit.control_plane.rs"] -mod codegen; +include!("../codegen/quickwit/quickwit.control_plane.rs"); -pub use codegen::*; - -pub type Result = std::result::Result; +pub type ControlPlaneResult = std::result::Result; #[derive(Debug, thiserror::Error)] pub enum ControlPlaneError { diff --git a/quickwit/quickwit-proto/src/indexing/mod.rs b/quickwit/quickwit-proto/src/indexing/mod.rs index 451d784f98a..ad3fb33e5b8 100644 --- a/quickwit/quickwit-proto/src/indexing/mod.rs +++ b/quickwit/quickwit-proto/src/indexing/mod.rs @@ -25,12 +25,9 @@ use thiserror; use crate::{IndexUid, ServiceError, ServiceErrorCode}; -#[path = "../codegen/quickwit/quickwit.indexing.rs"] -mod codegen; +include!("../codegen/quickwit/quickwit.indexing.rs"); -pub use codegen::*; - -pub type Result = std::result::Result; +pub type IndexingResult = std::result::Result; #[derive(Debug, thiserror::Error)] pub enum IndexingError { @@ -177,11 +174,13 @@ impl TryFrom<&str> for IndexingTask { Ok(IndexingTask { index_uid: format!("{part2}:{part1}"), source_id: source_id.to_string(), + // shard_ids: Vec::new(), }) } else { Ok(IndexingTask { index_uid: part1.to_string(), source_id: source_id.to_string(), + // shard_ids: Vec::new(), }) } } @@ -196,6 +195,7 @@ mod tests { let original = IndexingTask { index_uid: "test-index:123456".to_string(), source_id: "test-source".to_string(), + // shard_ids: Vec::new(), }; let serialized = original.to_string(); @@ -210,6 +210,7 @@ mod tests { IndexingTask { index_uid: "foo".to_string(), source_id: "bar".to_string(), + // shard_ids: Vec::new(), } ); } diff --git a/quickwit/quickwit-proto/src/ingest/ingester.rs b/quickwit/quickwit-proto/src/ingest/ingester.rs new file mode 100644 index 00000000000..fde27fa38f2 --- /dev/null +++ b/quickwit/quickwit-proto/src/ingest/ingester.rs @@ -0,0 +1,223 @@ +// 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 bytes::Bytes; + +use crate::types::{queue_id, QueueId}; + +include!("../codegen/quickwit/quickwit.ingest.ingester.rs"); + +pub use ingester_service_grpc_server::IngesterServiceGrpcServer; + +impl FetchResponseV2 { + pub fn queue_id(&self) -> QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) + } + + pub fn docs(&self) -> impl Iterator + '_ { + self.doc_batch.iter().flat_map(|doc_batch| doc_batch.docs()) + } + + pub fn num_docs(&self) -> usize { + if let Some(doc_batch) = &self.doc_batch { + doc_batch.doc_lengths.len() + } else { + 0 + } + } + + pub fn to_position_inclusive(&self) -> Option { + let Some(doc_batch) = &self.doc_batch else { + return None; + }; + let num_docs = doc_batch.num_docs() as u64; + Some(self.from_position_inclusive + num_docs - 1) + } +} + +impl OpenFetchStreamRequest { + pub fn queue_id(&self) -> QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) + } +} + +impl PersistSubrequest { + pub fn queue_id(&self) -> QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) + } +} + +impl PersistSuccess { + pub fn queue_id(&self) -> QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) + } +} + +impl SynReplicationMessage { + pub fn into_open_request(self) -> Option { + match self.message { + Some(syn_replication_message::Message::OpenRequest(open_request)) => Some(open_request), + _ => None, + } + } + + pub fn into_replicate_request(self) -> Option { + match self.message { + Some(syn_replication_message::Message::ReplicateRequest(replicate_request)) => { + Some(replicate_request) + } + _ => None, + } + } + + pub fn new_open_request(open_request: OpenReplicationStreamRequest) -> Self { + Self { + message: Some(syn_replication_message::Message::OpenRequest(open_request)), + } + } + + pub fn new_replicate_request(replicate_request: ReplicateRequest) -> Self { + Self { + message: Some(syn_replication_message::Message::ReplicateRequest( + replicate_request, + )), + } + } + + pub fn new_truncate_request(truncate_request: TruncateRequest) -> Self { + Self { + message: Some(syn_replication_message::Message::TruncateRequest( + truncate_request, + )), + } + } +} + +impl AckReplicationMessage { + pub fn into_open_response(self) -> Option { + match self.message { + Some(ack_replication_message::Message::OpenResponse(open_response)) => { + Some(open_response) + } + _ => None, + } + } + + pub fn new_open_response(open_response: OpenReplicationStreamResponse) -> Self { + Self { + message: Some(ack_replication_message::Message::OpenResponse( + open_response, + )), + } + } + + pub fn new_replicate_response(replicate_response: ReplicateResponse) -> Self { + Self { + message: Some(ack_replication_message::Message::ReplicateResponse( + replicate_response, + )), + } + } + + pub fn new_truncate_response(truncate_response: TruncateResponse) -> Self { + Self { + message: Some(ack_replication_message::Message::TruncateResponse( + truncate_response, + )), + } + } +} + +impl ReplicateSubrequest { + pub fn queue_id(&self) -> QueueId { + 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; + + match self.from_position_exclusive { + Some(from_position_exclusive) => Some(from_position_exclusive + num_docs), + None => Some(num_docs - 1), + } + } +} + +impl TruncateSubrequest { + pub fn queue_id(&self) -> QueueId { + queue_id(&self.index_uid, &self.source_id, self.shard_id) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::ingest::DocBatchV2; + + #[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, + doc_batch: None, + }; + assert_eq!(response.to_position_inclusive(), None); + + response.doc_batch = Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doc"), + doc_lengths: vec![8], + }); + assert_eq!(response.to_position_inclusive(), Some(0)); + + response.doc_batch = Some(DocBatchV2 { + doc_buffer: Bytes::from_static(b"test-doctest-doc"), + doc_lengths: vec![8, 8], + }); + 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)); + } +} diff --git a/quickwit/quickwit-proto/src/ingest/mod.rs b/quickwit/quickwit-proto/src/ingest/mod.rs new file mode 100644 index 00000000000..4a1dca64c40 --- /dev/null +++ b/quickwit/quickwit-proto/src/ingest/mod.rs @@ -0,0 +1,157 @@ +// 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 bytes::Bytes; + +use super::types::{NodeId, ShardId, SourceId}; +use super::{IndexUid, ServiceError, ServiceErrorCode}; +use crate::control_plane::ControlPlaneError; + +pub mod ingester; +pub mod router; + +include!("../codegen/quickwit/quickwit.ingest.rs"); + +pub type IngestV2Result = std::result::Result; + +#[derive(Debug, Clone, thiserror::Error)] +pub enum IngestV2Error { + #[error("An internal error occurred: {0}.")] + Internal(String), + #[error("Failed to connect to ingester `{ingester_id}`.")] + IngesterUnavailable { ingester_id: NodeId }, + #[error( + "Ingest service is currently unavailable with {num_ingesters} in the cluster and a \ + replication factor of {replication_factor}." + )] + ServiceUnavailable { + num_ingesters: usize, + replication_factor: usize, + }, + // #[error("Could not find shard.")] + // ShardNotFound { + // index_uid: IndexUid, + // source_id: SourceId, + // shard_id: ShardId, + // }, + #[error("Failed to open or write to shard.")] + ShardUnavailable { + leader_id: NodeId, + index_uid: IndexUid, + source_id: SourceId, + shard_id: ShardId, + }, +} + +impl From for IngestV2Error { + fn from(error: ControlPlaneError) -> Self { + Self::Internal(error.to_string()) + } +} + +impl From for tonic::Status { + fn from(error: IngestV2Error) -> tonic::Status { + let code = match &error { + IngestV2Error::Internal(_) => tonic::Code::Internal, + IngestV2Error::IngesterUnavailable { .. } => tonic::Code::Unavailable, + IngestV2Error::ShardUnavailable { .. } => tonic::Code::Unavailable, + IngestV2Error::ServiceUnavailable { .. } => tonic::Code::Unavailable, + }; + let message = error.to_string(); + tonic::Status::new(code, message) + } +} + +impl From for IngestV2Error { + fn from(status: tonic::Status) -> Self { + IngestV2Error::Internal(status.message().to_string()) + } +} + +impl ServiceError for IngestV2Error { + fn status_code(&self) -> ServiceErrorCode { + match self { + Self::Internal { .. } => ServiceErrorCode::Internal, + Self::IngesterUnavailable { .. } => ServiceErrorCode::Unavailable, + Self::ShardUnavailable { .. } => ServiceErrorCode::Unavailable, + Self::ServiceUnavailable { .. } => ServiceErrorCode::Unavailable, + } + } +} + +impl DocBatchV2 { + pub fn docs(&self) -> impl Iterator + '_ { + self.doc_lengths.iter().scan(0, |start_offset, doc_length| { + let start = *start_offset; + let end = start + *doc_length as usize; + *start_offset = end; + Some(self.doc_buffer.slice(start..end)) + }) + } + + pub fn is_empty(&self) -> bool { + self.doc_lengths.is_empty() + } + + pub fn num_bytes(&self) -> usize { + self.doc_buffer.len() + } + + pub fn num_docs(&self) -> usize { + self.doc_lengths.len() + } +} + +impl Shard { + pub fn is_closing(&self) -> bool { + self.shard_state() == ShardState::Closing + } + + pub fn is_closed(&self) -> bool { + self.shard_state() == ShardState::Closed + } + + pub fn is_open(&self) -> bool { + self.shard_state() == ShardState::Open + } + + pub fn is_deletable(&self) -> bool { + self.is_closed() + && self.publish_position_inclusive.parse::().ok() + >= self.replication_position_inclusive + } + + pub fn queue_id(&self) -> super::types::QueueId { + super::types::queue_id(&self.index_uid, &self.source_id, self.shard_id) + } +} + +impl ShardState { + pub fn is_closing(&self) -> bool { + *self == ShardState::Closing + } + + pub fn is_closed(&self) -> bool { + *self == ShardState::Closed + } + + pub fn is_open(&self) -> bool { + *self == ShardState::Open + } +} diff --git a/quickwit/quickwit-proto/src/ingest/router.rs b/quickwit/quickwit-proto/src/ingest/router.rs new file mode 100644 index 00000000000..0e831a72dbe --- /dev/null +++ b/quickwit/quickwit-proto/src/ingest/router.rs @@ -0,0 +1,28 @@ +// 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 super::Bytes; + +include!("../codegen/quickwit/quickwit.ingest.router.rs"); + +impl IngestSubrequest { + pub fn docs(&self) -> impl Iterator + '_ { + self.doc_batch.iter().flat_map(|doc_batch| doc_batch.docs()) + } +} diff --git a/quickwit/quickwit-proto/src/lib.rs b/quickwit/quickwit-proto/src/lib.rs index 6ecded6122a..861a1f94ef5 100644 --- a/quickwit/quickwit-proto/src/lib.rs +++ b/quickwit/quickwit-proto/src/lib.rs @@ -21,7 +21,6 @@ #![deny(clippy::disallowed_methods)] #![allow(rustdoc::invalid_html_tags)] -use std::cmp::Ordering; use std::convert::Infallible; use std::fmt; @@ -32,19 +31,16 @@ use tonic::service::Interceptor; use tonic::Status; use tracing::Span; use tracing_opentelemetry::OpenTelemetrySpanExt; -use ulid::Ulid; pub mod control_plane; +pub use {bytes, tonic}; pub mod indexing; -#[path = "codegen/quickwit/quickwit.metastore.rs"] +pub mod ingest; pub mod metastore; -#[path = "codegen/quickwit/quickwit.search.rs"] pub mod search; +pub mod types; -pub use metastore::*; -pub use search::*; -pub use sort_by_value::SortValue; -pub use tonic; +pub use types::*; pub mod jaeger { pub mod api_v2 { @@ -179,10 +175,10 @@ pub fn convert_to_grpc_result( .map_err(|error| error.grpc_error()) } -impl TryFrom for SearchRequest { +impl TryFrom for search::SearchRequest { type Error = anyhow::Error; - fn try_from(search_stream_req: SearchStreamRequest) -> Result { + fn try_from(search_stream_req: search::SearchStreamRequest) -> Result { Ok(Self { index_id: search_stream_req.index_id, query_ast: search_stream_req.query_ast, @@ -194,10 +190,10 @@ impl TryFrom for SearchRequest { } } -impl TryFrom for SearchRequest { +impl TryFrom for search::SearchRequest { type Error = anyhow::Error; - fn try_from(delete_query: DeleteQuery) -> anyhow::Result { + fn try_from(delete_query: metastore::DeleteQuery) -> anyhow::Result { let index_uid: IndexUid = delete_query.index_uid.into(); Ok(Self { index_id: index_uid.index_id().to_string(), @@ -209,34 +205,6 @@ impl TryFrom for SearchRequest { } } -impl SearchRequest { - pub fn time_range(&self) -> impl std::ops::RangeBounds { - use std::ops::Bound; - ( - self.start_timestamp - .map_or(Bound::Unbounded, Bound::Included), - self.end_timestamp.map_or(Bound::Unbounded, Bound::Excluded), - ) - } -} - -impl SplitIdAndFooterOffsets { - pub fn time_range(&self) -> impl std::ops::RangeBounds { - use std::ops::Bound; - ( - self.timestamp_start - .map_or(Bound::Unbounded, Bound::Included), - self.timestamp_end.map_or(Bound::Unbounded, Bound::Included), - ) - } -} - -impl fmt::Display for SplitSearchError { - fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "({}, split_id: {})", self.error, self.split_id) - } -} - /// `MutMetadataMap` used to extract [`tonic::metadata::MetadataMap`] from a request. pub struct MutMetadataMap<'a>(&'a mut tonic::metadata::MetadataMap); @@ -317,129 +285,6 @@ pub fn set_parent_span_from_request_metadata(request_metadata: &tonic::metadata: Span::current().set_parent(parent_cx); } -/// Index identifiers that uniquely identify not only the index, but also -/// its incarnation allowing to distinguish between deleted and recreated indexes. -/// It is represented as a stiring in index_id:incarnation_id format. -#[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq, Eq, Ord, PartialOrd, Hash)] -pub struct IndexUid(String); - -impl IndexUid { - /// Creates a new index uid form index_id and incarnation_id - pub fn new(index_id: impl Into) -> Self { - Self::from_parts(index_id, Ulid::new().to_string()) - } - - pub fn from_parts(index_id: impl Into, incarnation_id: impl Into) -> Self { - let incarnation_id = incarnation_id.into(); - let index_id = index_id.into(); - if incarnation_id.is_empty() { - Self(index_id) - } else { - Self(format!("{index_id}:{incarnation_id}")) - } - } - - pub fn index_id(&self) -> &str { - self.0.split(':').next().unwrap() - } - - pub fn incarnation_id(&self) -> &str { - if let Some(incarnation_id) = self.0.split(':').nth(1) { - incarnation_id - } else { - "" - } - } - - pub fn is_empty(&self) -> bool { - self.0.is_empty() - } -} - -impl From for String { - fn from(val: IndexUid) -> Self { - val.0 - } -} - -impl fmt::Display for IndexUid { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "{}", self.0) - } -} - -impl From for IndexUid { - fn from(index_uid: String) -> Self { - IndexUid(index_uid) - } -} - -// !!! Disclaimer !!! -// -// Prost imposes the PartialEq derived implementation. -// This is terrible because this means Eq, PartialEq are not really in line with Ord's -// implementation. if in presence of NaN. -impl Eq for SortByValue {} -impl Copy for SortByValue {} -impl From for SortByValue { - fn from(sort_value: SortValue) -> Self { - SortByValue { - sort_value: Some(sort_value), - } - } -} - -impl Copy for SortValue {} -impl Eq for SortValue {} - -impl Ord for SortValue { - fn cmp(&self, other: &Self) -> Ordering { - // We make sure to end up with a total order. - match (*self, *other) { - // Same types. - (SortValue::U64(left), SortValue::U64(right)) => left.cmp(&right), - (SortValue::I64(left), SortValue::I64(right)) => left.cmp(&right), - (SortValue::F64(left), SortValue::F64(right)) => { - if left.is_nan() { - if right.is_nan() { - Ordering::Equal - } else { - Ordering::Less - } - } else if right.is_nan() { - Ordering::Greater - } else { - left.partial_cmp(&right).unwrap_or(Ordering::Less) - } - } - (SortValue::Boolean(left), SortValue::Boolean(right)) => left.cmp(&right), - // We half the logic by making sure we keep - // the "stronger" type on the left. - (SortValue::U64(left), SortValue::I64(right)) => { - if left > i64::MAX as u64 { - return Ordering::Greater; - } - (left as i64).cmp(&right) - } - (SortValue::F64(left), _) if left.is_nan() => Ordering::Less, - (SortValue::F64(left), SortValue::U64(right)) => { - left.partial_cmp(&(right as f64)).unwrap_or(Ordering::Less) - } - (SortValue::F64(left), SortValue::I64(right)) => { - left.partial_cmp(&(right as f64)).unwrap_or(Ordering::Less) - } - (SortValue::Boolean(left), right) => SortValue::U64(left as u64).cmp(&right), - (left, right) => right.cmp(&left).reverse(), - } - } -} - -impl PartialOrd for SortValue { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - impl ServiceError for quickwit_actors::AskError { fn status_code(&self) -> ServiceErrorCode { match self { @@ -449,39 +294,3 @@ impl ServiceError for quickwit_actors::AskError } } } - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_index_uid_parsing() { - assert_eq!("foo", IndexUid::from("foo".to_string()).index_id()); - assert_eq!("foo", IndexUid::from("foo:bar".to_string()).index_id()); - assert_eq!("", IndexUid::from("foo".to_string()).incarnation_id()); - assert_eq!( - "bar", - IndexUid::from("foo:bar".to_string()).incarnation_id() - ); - } - - #[test] - fn test_index_uid_roundtrip() { - assert_eq!("foo", IndexUid::from("foo".to_string()).to_string()); - assert_eq!("foo:bar", IndexUid::from("foo:bar".to_string()).to_string()); - } - - #[test] - fn test_index_uid_roundtrip_using_parts() { - assert_eq!("foo", index_uid_roundtrip_using_parts("foo")); - assert_eq!("foo:bar", index_uid_roundtrip_using_parts("foo:bar")); - } - - fn index_uid_roundtrip_using_parts(index_uid: &str) -> String { - let index_uid = IndexUid::from(index_uid.to_string()); - let index_id = index_uid.index_id(); - let incarnation_id = index_uid.incarnation_id(); - let index_uid_from_parts = IndexUid::from_parts(index_id, incarnation_id); - index_uid_from_parts.to_string() - } -} diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs new file mode 100644 index 00000000000..24793f8c5ee --- /dev/null +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -0,0 +1,23 @@ +// 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 . + +include!("../codegen/quickwit/quickwit.metastore.rs"); + +pub use metastore_service_client::MetastoreServiceClient; +pub use metastore_service_server::{MetastoreService, MetastoreServiceServer}; diff --git a/quickwit/quickwit-proto/src/search/mod.rs b/quickwit/quickwit-proto/src/search/mod.rs new file mode 100644 index 00000000000..299d47584e5 --- /dev/null +++ b/quickwit/quickwit-proto/src/search/mod.rs @@ -0,0 +1,119 @@ +// 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::cmp::Ordering; +use std::fmt; + +pub use sort_by_value::SortValue; + +include!("../codegen/quickwit/quickwit.search.rs"); + +impl SearchRequest { + pub fn time_range(&self) -> impl std::ops::RangeBounds { + use std::ops::Bound; + ( + self.start_timestamp + .map_or(Bound::Unbounded, Bound::Included), + self.end_timestamp.map_or(Bound::Unbounded, Bound::Excluded), + ) + } +} + +impl SplitIdAndFooterOffsets { + pub fn time_range(&self) -> impl std::ops::RangeBounds { + use std::ops::Bound; + ( + self.timestamp_start + .map_or(Bound::Unbounded, Bound::Included), + self.timestamp_end.map_or(Bound::Unbounded, Bound::Included), + ) + } +} + +impl fmt::Display for SplitSearchError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "({}, split_id: {})", self.error, self.split_id) + } +} + +// !!! Disclaimer !!! +// +// Prost imposes the PartialEq derived implementation. +// This is terrible because this means Eq, PartialEq are not really in line with Ord's +// implementation. if in presence of NaN. +impl Eq for SortByValue {} +impl Copy for SortByValue {} +impl From for SortByValue { + fn from(sort_value: SortValue) -> Self { + SortByValue { + sort_value: Some(sort_value), + } + } +} + +impl Copy for SortValue {} +impl Eq for SortValue {} + +impl Ord for SortValue { + fn cmp(&self, other: &Self) -> Ordering { + // We make sure to end up with a total order. + match (*self, *other) { + // Same types. + (SortValue::U64(left), SortValue::U64(right)) => left.cmp(&right), + (SortValue::I64(left), SortValue::I64(right)) => left.cmp(&right), + (SortValue::F64(left), SortValue::F64(right)) => { + if left.is_nan() { + if right.is_nan() { + Ordering::Equal + } else { + Ordering::Less + } + } else if right.is_nan() { + Ordering::Greater + } else { + left.partial_cmp(&right).unwrap_or(Ordering::Less) + } + } + (SortValue::Boolean(left), SortValue::Boolean(right)) => left.cmp(&right), + // We half the logic by making sure we keep + // the "stronger" type on the left. + (SortValue::U64(left), SortValue::I64(right)) => { + if left > i64::MAX as u64 { + return Ordering::Greater; + } + (left as i64).cmp(&right) + } + (SortValue::F64(left), _) if left.is_nan() => Ordering::Less, + (SortValue::F64(left), SortValue::U64(right)) => { + left.partial_cmp(&(right as f64)).unwrap_or(Ordering::Less) + } + (SortValue::F64(left), SortValue::I64(right)) => { + left.partial_cmp(&(right as f64)).unwrap_or(Ordering::Less) + } + (SortValue::Boolean(left), right) => SortValue::U64(left as u64).cmp(&right), + (left, right) => right.cmp(&left).reverse(), + } + } +} + +impl PartialOrd for SortValue { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} diff --git a/quickwit/quickwit-proto/src/types.rs b/quickwit/quickwit-proto/src/types.rs new file mode 100644 index 00000000000..d7a6b8d5e9e --- /dev/null +++ b/quickwit/quickwit-proto/src/types.rs @@ -0,0 +1,313 @@ +// 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::borrow::Borrow; +use std::convert::Infallible; +use std::fmt; +use std::fmt::Display; +use std::ops::Deref; +use std::str::FromStr; + +use serde::{Deserialize, Serialize}; +use ulid::Ulid; + +pub type IndexId = String; + +pub type SourceId = String; + +pub type ShardId = u64; + +/// Uniquely identifies a shard and its underlying mrecordlog queue. +pub type QueueId = String; // // + +pub fn queue_id(index_uid: &str, source_id: &str, shard_id: u64) -> QueueId { + format!("{}/{}/{}", index_uid, source_id, shard_id) +} + +/// Index identifiers that uniquely identify not only the index, but also +/// its incarnation allowing to distinguish between deleted and recreated indexes. +/// It is represented as a stiring in index_id:incarnation_id format. +#[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq, Eq, Ord, PartialOrd, Hash)] +pub struct IndexUid(String); + +impl fmt::Display for IndexUid { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.0) + } +} + +impl IndexUid { + /// Creates a new index uid form index_id and incarnation_id + pub fn new(index_id: impl Into) -> Self { + Self::from_parts(index_id, Ulid::new().to_string()) + } + + pub fn as_str(&self) -> &str { + &self.0 + } + + pub fn from_parts(index_id: impl Into, incarnation_id: impl Into) -> Self { + let incarnation_id = incarnation_id.into(); + let index_id = index_id.into(); + if incarnation_id.is_empty() { + Self(index_id) + } else { + Self(format!("{index_id}:{incarnation_id}")) + } + } + + pub fn index_id(&self) -> &str { + self.0.split(':').next().unwrap() + } + + pub fn incarnation_id(&self) -> &str { + if let Some(incarnation_id) = self.0.split(':').nth(1) { + incarnation_id + } else { + "" + } + } + + pub fn is_empty(&self) -> bool { + self.0.is_empty() + } +} + +impl From for String { + fn from(val: IndexUid) -> Self { + val.0 + } +} + +impl From<&str> for IndexUid { + fn from(index_uid: &str) -> Self { + Self(index_uid.to_string()) + } +} + +impl From for IndexUid { + fn from(index_uid: String) -> Self { + Self(index_uid) + } +} + +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash, Serialize, Deserialize)] +pub struct NodeId(String); + +// #[derive(Debug, Clone, Copy, PartialEq, Eq, PartialOrd, Ord, Hash)] +// pub struct GenerationId(u64); + +// #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +// pub struct NodeUid(NodeId, GenerationId); + +impl NodeId { + /// Constructs a new [`NodeId`]. + pub const fn new(node_id: String) -> Self { + Self(node_id) + } + + /// Takes ownership of the underlying [`String`], consuming `self`. + pub fn take(self) -> String { + self.0 + } +} + +impl AsRef for NodeId { + fn as_ref(&self) -> &str { + self.as_str() + } +} + +impl AsRef for NodeId { + fn as_ref(&self) -> &NodeIdRef { + self.deref() + } +} + +impl Borrow for NodeId { + fn borrow(&self) -> &str { + self.as_str() + } +} + +impl Borrow for NodeId { + fn borrow(&self) -> &NodeIdRef { + self.deref() + } +} + +impl Deref for NodeId { + type Target = NodeIdRef; + + fn deref(&self) -> &Self::Target { + NodeIdRef::from_str(&self.0) + } +} + +impl Display for NodeId { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0) + } +} + +impl From<&'_ str> for NodeId { + fn from(node_id: &str) -> Self { + Self::new(node_id.to_string()) + } +} + +impl From for NodeId { + fn from(node_id: String) -> Self { + Self::new(node_id) + } +} + +impl From for String { + fn from(node_id: NodeId) -> Self { + node_id.0 + } +} + +impl From<&'_ NodeIdRef> for NodeId { + fn from(node_id: &NodeIdRef) -> Self { + node_id.to_owned() + } +} + +impl FromStr for NodeId { + type Err = Infallible; + + fn from_str(node_id: &str) -> Result { + Ok(NodeId::new(node_id.to_string())) + } +} + +impl PartialEq<&str> for NodeId { + fn eq(&self, other: &&str) -> bool { + self.as_str() == *other + } +} + +#[repr(transparent)] +#[derive(Debug, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct NodeIdRef(str); + +impl NodeIdRef { + /// Transparently reinterprets the string slice as a strongly-typed [`NodeIdRef`]. + pub const fn from_str(node_id: &str) -> &Self { + let ptr: *const str = node_id; + // SAFETY: `NodeIdRef` is `#[repr(transparent)]` around a single `str` field, so a `*const + // str` can be safely reinterpreted as a `*const NodeIdRef` + unsafe { &*(ptr as *const Self) } + } + + /// Transparently reinterprets the static string slice as a strongly-typed [`NodeIdRef`]. + pub const fn from_static(node_id: &'static str) -> &'static Self { + Self::from_str(node_id) + } + + /// Provides access to the underlying value as a string slice. + pub const fn as_str(&self) -> &str { + &self.0 + } +} + +impl AsRef for NodeIdRef { + fn as_ref(&self) -> &str { + &self.0 + } +} + +impl Borrow for NodeIdRef { + fn borrow(&self) -> &str { + &self.0 + } +} + +impl<'a> From<&'a str> for &'a NodeIdRef { + fn from(node_id: &'a str) -> &'a NodeIdRef { + NodeIdRef::from_str(node_id) + } +} + +impl PartialEq for NodeId { + fn eq(&self, other: &NodeIdRef) -> bool { + self.as_str() == other.as_str() + } +} + +impl PartialEq<&'_ NodeIdRef> for NodeId { + fn eq(&self, other: &&NodeIdRef) -> bool { + self.as_str() == other.as_str() + } +} + +impl PartialEq for NodeIdRef { + fn eq(&self, other: &NodeId) -> bool { + self.as_str() == other.as_str() + } +} + +impl PartialEq for &'_ NodeIdRef { + fn eq(&self, other: &NodeId) -> bool { + self.as_str() == other.as_str() + } +} + +impl PartialEq for String { + fn eq(&self, other: &NodeId) -> bool { + self.as_str() == other.as_str() + } +} + +impl ToOwned for NodeIdRef { + type Owned = NodeId; + + fn to_owned(&self) -> Self::Owned { + NodeId(self.0.to_string()) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_node_id() { + let node_id = NodeId::new("test-node".to_string()); + assert_eq!(node_id.as_str(), "test-node"); + assert_eq!(node_id, NodeIdRef::from_str("test-node")); + } + + #[test] + fn test_node_serde() { + #[derive(Debug, PartialEq, Eq, Serialize, Deserialize)] + struct Node { + node_id: NodeId, + } + let node = Node { + node_id: NodeId::from("test-node"), + }; + let serialized = serde_json::to_string(&node).unwrap(); + assert_eq!(serialized, r#"{"node_id":"test-node"}"#); + + let deserialized = serde_json::from_str::(&serialized).unwrap(); + assert_eq!(deserialized, node); + } +} diff --git a/quickwit/quickwit-search/src/client.rs b/quickwit/quickwit-search/src/client.rs index 9b2aa32e4ea..6d2b22bb6bb 100644 --- a/quickwit/quickwit-search/src/client.rs +++ b/quickwit/quickwit-search/src/client.rs @@ -24,14 +24,12 @@ use std::time::Duration; use futures::{StreamExt, TryStreamExt}; use http::Uri; +use quickwit_proto::search::{GetKvRequest, LeafSearchStreamResponse, PutKvRequest}; use quickwit_proto::tonic::codegen::InterceptedService; -use quickwit_proto::tonic::transport::Endpoint; -use quickwit_proto::{ - tonic, GetKvRequest, LeafSearchStreamResponse, PutKvRequest, SpanContextInterceptor, -}; +use quickwit_proto::tonic::transport::{Channel, Endpoint}; +use quickwit_proto::tonic::Request; +use quickwit_proto::{tonic, SpanContextInterceptor}; use tokio_stream::wrappers::UnboundedReceiverStream; -use tonic::transport::Channel; -use tonic::Request; use tower::timeout::Timeout; use tracing::*; @@ -43,7 +41,7 @@ use crate::SearchService; enum SearchServiceClientImpl { Local(Arc), Grpc( - quickwit_proto::search_service_client::SearchServiceClient< + quickwit_proto::search::search_service_client::SearchServiceClient< InterceptedService, SpanContextInterceptor>, >, ), @@ -74,7 +72,7 @@ impl fmt::Debug for SearchServiceClient { impl SearchServiceClient { /// Create a search service client instance given a gRPC client and gRPC address. pub fn from_grpc_client( - client: quickwit_proto::search_service_client::SearchServiceClient< + client: quickwit_proto::search::search_service_client::SearchServiceClient< InterceptedService, SpanContextInterceptor>, >, grpc_addr: SocketAddr, @@ -107,8 +105,8 @@ impl SearchServiceClient { /// Perform root search. pub async fn root_search( &mut self, - request: quickwit_proto::SearchRequest, - ) -> crate::Result { + request: quickwit_proto::search::SearchRequest, + ) -> crate::Result { match &mut self.client_impl { SearchServiceClientImpl::Grpc(grpc_client) => { let tonic_request = Request::new(request); @@ -125,8 +123,8 @@ impl SearchServiceClient { /// Perform leaf search. pub async fn leaf_search( &mut self, - request: quickwit_proto::LeafSearchRequest, - ) -> crate::Result { + request: quickwit_proto::search::LeafSearchRequest, + ) -> crate::Result { match &mut self.client_impl { SearchServiceClientImpl::Grpc(grpc_client) => { let tonic_request = Request::new(request); @@ -143,7 +141,7 @@ impl SearchServiceClient { /// Perform leaf stream. pub async fn leaf_search_stream( &mut self, - request: quickwit_proto::LeafSearchStreamRequest, + request: quickwit_proto::search::LeafSearchStreamRequest, ) -> UnboundedReceiverStream> { match &mut self.client_impl { SearchServiceClientImpl::Grpc(grpc_client) => { @@ -197,8 +195,8 @@ impl SearchServiceClient { /// Perform fetch docs. pub async fn fetch_docs( &mut self, - request: quickwit_proto::FetchDocsRequest, - ) -> crate::Result { + request: quickwit_proto::search::FetchDocsRequest, + ) -> crate::Result { match &mut self.client_impl { SearchServiceClientImpl::Grpc(grpc_client) => { let tonic_request = Request::new(request); @@ -215,8 +213,8 @@ impl SearchServiceClient { /// Perform leaf list terms. pub async fn leaf_list_terms( &mut self, - request: quickwit_proto::LeafListTermsRequest, - ) -> crate::Result { + request: quickwit_proto::search::LeafListTermsRequest, + ) -> crate::Result { match &mut self.client_impl { SearchServiceClientImpl::Grpc(grpc_client) => { let tonic_request = Request::new(request); @@ -240,7 +238,7 @@ impl SearchServiceClient { Ok(search_after_context_opt) } SearchServiceClientImpl::Grpc(grpc_client) => { - let grpc_resp: tonic::Response = grpc_client + let grpc_resp: tonic::Response = grpc_client .get_kv(get_kv_req) .await .map_err(|tonic_error| parse_grpc_error(&tonic_error))?; @@ -281,10 +279,11 @@ pub fn create_search_client_from_grpc_addr(grpc_addr: SocketAddr) -> SearchServi .expect("The URI should be well-formed."); let channel = Endpoint::from(uri).connect_lazy(); let timeout_channel = Timeout::new(channel, Duration::from_secs(5)); - let client = quickwit_proto::search_service_client::SearchServiceClient::with_interceptor( - timeout_channel, - SpanContextInterceptor, - ); + let client = + quickwit_proto::search::search_service_client::SearchServiceClient::with_interceptor( + timeout_channel, + SpanContextInterceptor, + ); SearchServiceClient::from_grpc_client(client, grpc_addr) } @@ -293,9 +292,10 @@ pub fn create_search_client_from_channel( grpc_addr: SocketAddr, channel: Timeout, ) -> SearchServiceClient { - let client = quickwit_proto::search_service_client::SearchServiceClient::with_interceptor( - channel, - SpanContextInterceptor, - ); + let client = + quickwit_proto::search::search_service_client::SearchServiceClient::with_interceptor( + channel, + SpanContextInterceptor, + ); SearchServiceClient::from_grpc_client(client, grpc_addr) } diff --git a/quickwit/quickwit-search/src/cluster_client.rs b/quickwit/quickwit-search/src/cluster_client.rs index 664ebc895e2..73bb64660bf 100644 --- a/quickwit/quickwit-search/src/cluster_client.rs +++ b/quickwit/quickwit-search/src/cluster_client.rs @@ -22,7 +22,7 @@ use std::time::Duration; use base64::Engine; use futures::future::ready; use futures::{Future, StreamExt}; -use quickwit_proto::{ +use quickwit_proto::search::{ FetchDocsRequest, FetchDocsResponse, GetKvRequest, LeafListTermsRequest, LeafListTermsResponse, LeafSearchRequest, LeafSearchResponse, LeafSearchStreamRequest, LeafSearchStreamResponse, PutKvRequest, @@ -341,7 +341,7 @@ mod tests { use std::collections::HashSet; use std::net::SocketAddr; - use quickwit_proto::{ + use quickwit_proto::search::{ PartialHit, SearchRequest, SearchStreamRequest, SortValue, SplitIdAndFooterOffsets, SplitSearchError, }; @@ -446,8 +446,8 @@ mod tests { let request = mock_doc_request("split_1"); let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_fetch_docs().return_once( - |_: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { hits: Vec::new() }) + |_: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: Vec::new() }) }, ); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", mock_search_service)]); @@ -469,14 +469,14 @@ mod tests { let request = mock_doc_request("split_1"); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_fetch_docs().return_once( - |_: quickwit_proto::FetchDocsRequest| { + |_: quickwit_proto::search::FetchDocsRequest| { Err(SearchError::InternalError("error".to_string())) }, ); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_fetch_docs().return_once( - |_: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { hits: Vec::new() }) + |_: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: Vec::new() }) }, ); let searcher_pool = searcher_pool_for_test([ @@ -498,11 +498,11 @@ mod tests { async fn test_cluster_client_fetch_docs_retry_with_final_error() { let request = mock_doc_request("split_1"); let mut mock_search_service = MockSearchService::new(); - mock_search_service - .expect_fetch_docs() - .returning(|_: quickwit_proto::FetchDocsRequest| { + mock_search_service.expect_fetch_docs().returning( + |_: quickwit_proto::search::FetchDocsRequest| { Err(SearchError::InternalError("error".to_string())) - }); + }, + ); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1001", mock_search_service)]); let first_client_addr: SocketAddr = "127.0.0.1:1001".parse().unwrap(); let first_client = searcher_pool.get(&first_client_addr).await.unwrap(); @@ -711,22 +711,21 @@ mod tests { mock_search_service_1 .expect_put_kv() .once() - .returning(|_put_req: quickwit_proto::PutKvRequest| {}); - mock_search_service_1 - .expect_get_kv() - .once() - .returning(|_get_req: quickwit_proto::GetKvRequest| Some(b"my_payload".to_vec())); + .returning(|_put_req: quickwit_proto::search::PutKvRequest| {}); + mock_search_service_1.expect_get_kv().once().returning( + |_get_req: quickwit_proto::search::GetKvRequest| Some(b"my_payload".to_vec()), + ); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2 .expect_put_kv() .once() - .returning(|_put_req: quickwit_proto::PutKvRequest| {}); + .returning(|_put_req: quickwit_proto::search::PutKvRequest| {}); let mut mock_search_service_3 = MockSearchService::new(); // Due to the buffered call it is possible for the // put request to 3 to be emitted too. mock_search_service_3 .expect_put_kv() - .returning(|_put_req: quickwit_proto::PutKvRequest| {}); + .returning(|_put_req: quickwit_proto::search::PutKvRequest| {}); let searcher_pool = searcher_pool_for_test([ ("127.0.0.1:1001", mock_search_service_1), ("127.0.0.1:1002", mock_search_service_2), @@ -757,24 +756,23 @@ mod tests { mock_search_service_1 .expect_put_kv() .once() - .returning(|_put_req: quickwit_proto::PutKvRequest| {}); + .returning(|_put_req: quickwit_proto::search::PutKvRequest| {}); mock_search_service_1 .expect_get_kv() .once() - .returning(|_get_req: quickwit_proto::GetKvRequest| None); + .returning(|_get_req: quickwit_proto::search::GetKvRequest| None); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2 .expect_put_kv() .once() - .returning(|_put_req: quickwit_proto::PutKvRequest| {}); - mock_search_service_2 - .expect_get_kv() - .once() - .returning(|_get_req: quickwit_proto::GetKvRequest| Some(b"my_payload".to_vec())); + .returning(|_put_req: quickwit_proto::search::PutKvRequest| {}); + mock_search_service_2.expect_get_kv().once().returning( + |_get_req: quickwit_proto::search::GetKvRequest| Some(b"my_payload".to_vec()), + ); let mut mock_search_service_3 = MockSearchService::new(); mock_search_service_3 .expect_put_kv() - .returning(|_leaf_search_req: quickwit_proto::PutKvRequest| {}); + .returning(|_leaf_search_req: quickwit_proto::search::PutKvRequest| {}); let searcher_pool = searcher_pool_for_test([ ("127.0.0.1:1001", mock_search_service_1), ("127.0.0.1:1002", mock_search_service_2), diff --git a/quickwit/quickwit-search/src/collector.rs b/quickwit/quickwit-search/src/collector.rs index 5200fcd8358..677a7acdaa0 100644 --- a/quickwit/quickwit-search/src/collector.rs +++ b/quickwit/quickwit-search/src/collector.rs @@ -23,7 +23,7 @@ use std::collections::{BinaryHeap, HashSet}; use itertools::Itertools; use quickwit_common::binary_heap::top_k; use quickwit_doc_mapper::{DocMapper, WarmupInfo}; -use quickwit_proto::{LeafSearchResponse, PartialHit, SearchRequest, SortOrder, SortValue}; +use quickwit_proto::search::{LeafSearchResponse, PartialHit, SearchRequest, SortOrder, SortValue}; use serde::Deserialize; use tantivy::aggregation::agg_req::{get_fast_field_names, Aggregations}; use tantivy::aggregation::intermediate_agg_result::IntermediateAggregationResults; @@ -446,10 +446,10 @@ impl SegmentCollector for QuickwitSegmentCollector { sort_by.recover_typed_sort_value(hit.sort_value_opt1, hit.sort_value_opt2); PartialHit { - sort_value: Some(quickwit_proto::SortByValue { + sort_value: Some(quickwit_proto::search::SortByValue { sort_value: sort_value1, }), - sort_value2: Some(quickwit_proto::SortByValue { + sort_value2: Some(quickwit_proto::search::SortByValue { sort_value: sort_value2, }), segment_ord, @@ -885,7 +885,7 @@ pub(crate) fn make_merge_collector( mod tests { use std::cmp::Ordering; - use quickwit_proto::{PartialHit, SortOrder, SortValue}; + use quickwit_proto::search::{PartialHit, SortOrder, SortValue}; use super::PartialHitHeapItem; use crate::collector::top_k_partial_hits; diff --git a/quickwit/quickwit-search/src/fetch_docs.rs b/quickwit/quickwit-search/src/fetch_docs.rs index 4cbd9f11b44..0a72104994c 100644 --- a/quickwit/quickwit-search/src/fetch_docs.rs +++ b/quickwit/quickwit-search/src/fetch_docs.rs @@ -24,7 +24,9 @@ use anyhow::{Context, Ok}; use futures::{StreamExt, TryStreamExt}; use itertools::Itertools; use quickwit_doc_mapper::DocMapper; -use quickwit_proto::{FetchDocsResponse, PartialHit, SnippetRequest, SplitIdAndFooterOffsets}; +use quickwit_proto::search::{ + FetchDocsResponse, PartialHit, SnippetRequest, SplitIdAndFooterOffsets, +}; use quickwit_storage::Storage; use tantivy::query::Query; use tantivy::schema::{Field, Value}; @@ -129,13 +131,13 @@ pub async fn fetch_docs( ) .await?; - let hits: Vec = partial_hits + let hits: Vec = partial_hits .iter() .flat_map(|partial_hit| { let global_doc_addr = GlobalDocAddress::from_partial_hit(partial_hit); if let Some((_, document)) = global_doc_addr_to_doc_json.remove_entry(&global_doc_addr) { - Some(quickwit_proto::LeafHit { + Some(quickwit_proto::search::LeafHit { leaf_json: document.content_json, partial_hit: Some(partial_hit.clone()), leaf_snippet_json: document.snippet_json, diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 049a3da4951..1a18318665e 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -27,7 +27,7 @@ use futures::future::try_join_all; use itertools::{Either, Itertools}; use quickwit_directories::{CachingDirectory, HotDirectory, StorageDirectory}; use quickwit_doc_mapper::{DocMapper, TermRange, WarmupInfo}; -use quickwit_proto::{ +use quickwit_proto::search::{ LeafListTermsResponse, LeafSearchResponse, ListTermsRequest, SearchRequest, SplitIdAndFooterOffsets, SplitSearchError, }; diff --git a/quickwit/quickwit-search/src/leaf_cache.rs b/quickwit/quickwit-search/src/leaf_cache.rs index 6e0055f1346..2b31695ceec 100644 --- a/quickwit/quickwit-search/src/leaf_cache.rs +++ b/quickwit/quickwit-search/src/leaf_cache.rs @@ -20,7 +20,7 @@ use std::ops::Bound; use prost::Message; -use quickwit_proto::{LeafSearchResponse, SearchRequest, SplitIdAndFooterOffsets}; +use quickwit_proto::search::{LeafSearchResponse, SearchRequest, SplitIdAndFooterOffsets}; use quickwit_storage::{MemorySizedCache, OwnedBytes}; /// A cache to memoize `leaf_search_single_split` results. @@ -170,7 +170,7 @@ impl std::ops::RangeBounds for Range { #[cfg(test)] mod tests { - use quickwit_proto::{ + use quickwit_proto::search::{ LeafSearchResponse, PartialHit, SearchRequest, SortValue, SplitIdAndFooterOffsets, }; diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index c9010aa84b4..586e0e99d37 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -62,9 +62,8 @@ pub use find_trace_ids_collector::FindTraceIdsCollector; use quickwit_config::SearcherConfig; use quickwit_doc_mapper::tag_pruning::extract_tags_from_query; use quickwit_metastore::{ListSplitsQuery, Metastore, SplitMetadata, SplitState}; -use quickwit_proto::{ - IndexUid, PartialHit, SearchRequest, SearchResponse, SplitIdAndFooterOffsets, -}; +use quickwit_proto::search::{PartialHit, SearchRequest, SearchResponse, SplitIdAndFooterOffsets}; +use quickwit_proto::IndexUid; use quickwit_storage::StorageResolver; use tantivy::DocAddress; diff --git a/quickwit/quickwit-search/src/retry/mod.rs b/quickwit/quickwit-search/src/retry/mod.rs index 145118018d8..3fe6dc1f706 100644 --- a/quickwit/quickwit-search/src/retry/mod.rs +++ b/quickwit/quickwit-search/src/retry/mod.rs @@ -85,7 +85,7 @@ mod tests { use std::net::SocketAddr; use std::sync::Arc; - use quickwit_proto::{FetchDocsResponse, SplitIdAndFooterOffsets}; + use quickwit_proto::search::{FetchDocsResponse, SplitIdAndFooterOffsets}; use crate::retry::{retry_client, DefaultRetryPolicy, RetryPolicy}; use crate::{ diff --git a/quickwit/quickwit-search/src/retry/search.rs b/quickwit/quickwit-search/src/retry/search.rs index d60714f7265..96dde43bc5a 100644 --- a/quickwit/quickwit-search/src/retry/search.rs +++ b/quickwit/quickwit-search/src/retry/search.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 quickwit_proto::{LeafSearchRequest, LeafSearchResponse}; +use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use super::RetryPolicy; use crate::SearchError; @@ -54,7 +54,7 @@ impl RetryPolicy for LeafSea #[cfg(test)] mod tests { - use quickwit_proto::{ + use quickwit_proto::search::{ LeafSearchRequest, LeafSearchResponse, SearchRequest, SplitIdAndFooterOffsets, SplitSearchError, }; diff --git a/quickwit/quickwit-search/src/retry/search_stream.rs b/quickwit/quickwit-search/src/retry/search_stream.rs index 1c667aa496a..02ddbd6bb72 100644 --- a/quickwit/quickwit-search/src/retry/search_stream.rs +++ b/quickwit/quickwit-search/src/retry/search_stream.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 quickwit_proto::{LeafSearchStreamRequest, LeafSearchStreamResponse}; +use quickwit_proto::search::{LeafSearchStreamRequest, LeafSearchStreamResponse}; use tokio::sync::mpsc::error::SendError; use tracing::warn; @@ -75,7 +75,7 @@ impl #[cfg(test)] mod tests { - use quickwit_proto::{ + use quickwit_proto::search::{ LeafSearchStreamRequest, LeafSearchStreamResponse, SplitIdAndFooterOffsets, }; use tokio::sync::mpsc::error::SendError; diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 1db04f8bfdc..3fc81ea9450 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -29,7 +29,7 @@ use quickwit_common::uri::Uri; use quickwit_config::{build_doc_mapper, IndexConfig}; use quickwit_doc_mapper::{DocMapper, DYNAMIC_FIELD_NAME}; use quickwit_metastore::{Metastore, SplitMetadata}; -use quickwit_proto::{ +use quickwit_proto::search::{ FetchDocsRequest, FetchDocsResponse, Hit, LeafHit, LeafListTermsRequest, LeafListTermsResponse, LeafSearchRequest, LeafSearchResponse, ListTermsRequest, ListTermsResponse, PartialHit, SearchRequest, SearchResponse, SnippetRequest, SortField, SplitIdAndFooterOffsets, @@ -991,7 +991,7 @@ mod tests { use quickwit_config::SearcherConfig; use quickwit_indexing::mock_split; use quickwit_metastore::{IndexMetadata, MockMetastore}; - use quickwit_proto::{ScrollRequest, SortOrder, SortValue, SplitSearchError}; + use quickwit_proto::search::{ScrollRequest, SortOrder, SortValue, SplitSearchError}; use quickwit_query::query_ast::qast_helper; use tantivy::schema::{FAST, STORED, TEXT}; @@ -1035,8 +1035,8 @@ mod tests { split_id: &str, sort_value: u64, doc_id: u32, - ) -> quickwit_proto::PartialHit { - quickwit_proto::PartialHit { + ) -> quickwit_proto::search::PartialHit { + quickwit_proto::search::PartialHit { sort_value: Some(SortValue::U64(sort_value).into()), sort_value2: None, split_id: split_id.to_string(), @@ -1046,12 +1046,12 @@ mod tests { } fn get_doc_for_fetch_req( - fetch_docs_req: quickwit_proto::FetchDocsRequest, - ) -> Vec { + fetch_docs_req: quickwit_proto::search::FetchDocsRequest, + ) -> Vec { fetch_docs_req .partial_hits .into_iter() - .map(|req| quickwit_proto::LeafHit { + .map(|req| quickwit_proto::search::LeafHit { leaf_json: serde_json::to_string_pretty(&serde_json::json!({ "title": [req.doc_id.to_string()], "body": ["test 1"], @@ -1066,7 +1066,7 @@ mod tests { #[tokio::test] async fn test_root_search_offset_out_of_bounds_1085() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1087,8 +1087,8 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1"), mock_split("split2")])); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 3, partial_hits: vec![ mock_partial_hit("split1", 3, 1), @@ -1102,16 +1102,16 @@ mod tests { }, ); mock_search_service_2.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, ); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 2, partial_hits: vec![ mock_partial_hit("split2", 3, 1), @@ -1124,8 +1124,8 @@ mod tests { }, ); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1152,7 +1152,7 @@ mod tests { #[tokio::test] async fn test_root_search_single_split() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1172,8 +1172,8 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1")])); let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 3, partial_hits: vec![ mock_partial_hit("split1", 3, 1), @@ -1187,8 +1187,8 @@ mod tests { }, ); mock_search_service.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1212,7 +1212,7 @@ mod tests { #[tokio::test] async fn test_root_search_multiple_splits() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1232,8 +1232,8 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1"), mock_split("split2")])); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 2, partial_hits: vec![ mock_partial_hit("split1", 3, 1), @@ -1246,16 +1246,16 @@ mod tests { }, ); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, ); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 1, partial_hits: vec![mock_partial_hit("split2", 2, 2)], failed_splits: Vec::new(), @@ -1265,8 +1265,8 @@ mod tests { }, ); mock_search_service_2.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1293,7 +1293,7 @@ mod tests { #[tokio::test] async fn test_root_search_multiple_splits_sort_heteregeneous_field_ascending( ) -> anyhow::Result<()> { - let mut search_request = quickwit_proto::SearchRequest { + let mut search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1316,18 +1316,18 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1"), mock_split("split2")])); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 2, partial_hits: vec![ - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: Some(SortValue::U64(2u64).into()), sort_value2: None, split_id: "split1".to_string(), segment_ord: 0, doc_id: 0, }, - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: None, sort_value2: None, split_id: "split1".to_string(), @@ -1342,33 +1342,33 @@ mod tests { }, ); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, ); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 3, partial_hits: vec![ - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: Some(SortValue::I64(-1i64).into()), sort_value2: None, split_id: "split2".to_string(), segment_ord: 0, doc_id: 1, }, - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: Some(SortValue::I64(1i64).into()), sort_value2: None, split_id: "split2".to_string(), segment_ord: 0, doc_id: 0, }, - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: None, sort_value2: None, split_id: "split2".to_string(), @@ -1383,8 +1383,8 @@ mod tests { }, ); mock_search_service_2.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1461,7 +1461,7 @@ mod tests { #[tokio::test] async fn test_root_search_multiple_splits_sort_heteregeneous_field_descending( ) -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1481,18 +1481,18 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1"), mock_split("split2")])); let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 2, partial_hits: vec![ - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: Some(SortValue::U64(2u64).into()), sort_value2: None, split_id: "split1".to_string(), segment_ord: 0, doc_id: 0, }, - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: None, sort_value2: None, split_id: "split1".to_string(), @@ -1507,33 +1507,33 @@ mod tests { }, ); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, ); let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( - |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 3, partial_hits: vec![ - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: Some(SortValue::I64(1i64).into()), sort_value2: None, split_id: "split2".to_string(), segment_ord: 0, doc_id: 0, }, - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: Some(SortValue::I64(-1i64).into()), sort_value2: None, split_id: "split2".to_string(), segment_ord: 0, doc_id: 1, }, - quickwit_proto::PartialHit { + quickwit_proto::search::PartialHit { sort_value: None, sort_value2: None, split_id: "split2".to_string(), @@ -1548,8 +1548,8 @@ mod tests { }, ); mock_search_service_2.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1625,7 +1625,7 @@ mod tests { #[tokio::test] async fn test_root_search_multiple_splits_retry_on_other_node() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1648,23 +1648,25 @@ mod tests { mock_search_service_1 .expect_leaf_search() .times(1) - .returning(|_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { - // requests from split 2 arrive here - simulate failure - num_hits: 0, - partial_hits: Vec::new(), - failed_splits: vec![SplitSearchError { - error: "mock_error".to_string(), - split_id: "split2".to_string(), - retryable_error: true, - }], - num_attempted_splits: 1, - ..Default::default() - }) - }); + .returning( + |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { + // requests from split 2 arrive here - simulate failure + num_hits: 0, + partial_hits: Vec::new(), + failed_splits: vec![SplitSearchError { + error: "mock_error".to_string(), + split_id: "split2".to_string(), + retryable_error: true, + }], + num_attempted_splits: 1, + ..Default::default() + }) + }, + ); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1673,39 +1675,41 @@ mod tests { mock_search_service_2 .expect_leaf_search() .times(2) - .returning(|leaf_search_req: quickwit_proto::LeafSearchRequest| { - let split_ids: Vec<&str> = leaf_search_req - .split_offsets - .iter() - .map(|metadata| metadata.split_id.as_str()) - .collect(); - if split_ids == ["split1"] { - Ok(quickwit_proto::LeafSearchResponse { - num_hits: 2, - partial_hits: vec![ - mock_partial_hit("split1", 3, 1), - mock_partial_hit("split1", 1, 3), - ], - failed_splits: Vec::new(), - num_attempted_splits: 1, - ..Default::default() - }) - } else if split_ids == ["split2"] { - // RETRY REQUEST! - Ok(quickwit_proto::LeafSearchResponse { - num_hits: 1, - partial_hits: vec![mock_partial_hit("split2", 2, 2)], - failed_splits: Vec::new(), - num_attempted_splits: 1, - ..Default::default() - }) - } else { - panic!("unexpected request in test {split_ids:?}"); - } - }); + .returning( + |leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + let split_ids: Vec<&str> = leaf_search_req + .split_offsets + .iter() + .map(|metadata| metadata.split_id.as_str()) + .collect(); + if split_ids == ["split1"] { + Ok(quickwit_proto::search::LeafSearchResponse { + num_hits: 2, + partial_hits: vec![ + mock_partial_hit("split1", 3, 1), + mock_partial_hit("split1", 1, 3), + ], + failed_splits: Vec::new(), + num_attempted_splits: 1, + ..Default::default() + }) + } else if split_ids == ["split2"] { + // RETRY REQUEST! + Ok(quickwit_proto::search::LeafSearchResponse { + num_hits: 1, + partial_hits: vec![mock_partial_hit("split2", 2, 2)], + failed_splits: Vec::new(), + num_attempted_splits: 1, + ..Default::default() + }) + } else { + panic!("unexpected request in test {split_ids:?}"); + } + }, + ); mock_search_service_2.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1731,7 +1735,7 @@ mod tests { #[tokio::test] async fn test_root_search_multiple_splits_retry_on_all_nodes() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1756,7 +1760,7 @@ mod tests { .return_once(|_| { // requests from split 2 arrive here - simulate failure. // a retry will be made on the second service. - Ok(quickwit_proto::LeafSearchResponse { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 0, partial_hits: Vec::new(), failed_splits: vec![SplitSearchError { @@ -1773,7 +1777,7 @@ mod tests { .withf(|leaf_search_req| leaf_search_req.split_offsets[0].split_id == "split1") .return_once(|_| { // RETRY REQUEST from split1 - Ok(quickwit_proto::LeafSearchResponse { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 2, partial_hits: vec![ mock_partial_hit("split1", 3, 1), @@ -1785,8 +1789,8 @@ mod tests { }) }); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1797,7 +1801,7 @@ mod tests { .withf(|leaf_search_req| leaf_search_req.split_offsets[0].split_id == "split2") .return_once(|_| { // retry for split 2 arrive here, simulate success. - Ok(quickwit_proto::LeafSearchResponse { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 1, partial_hits: vec![mock_partial_hit("split2", 2, 2)], failed_splits: Vec::new(), @@ -1810,7 +1814,7 @@ mod tests { .withf(|leaf_search_req| leaf_search_req.split_offsets[0].split_id == "split1") .return_once(|_| { // requests from split 1 arrive here - simulate failure, then success. - Ok(quickwit_proto::LeafSearchResponse { + Ok(quickwit_proto::search::LeafSearchResponse { // requests from split 2 arrive here - simulate failure num_hits: 0, partial_hits: Vec::new(), @@ -1824,8 +1828,8 @@ mod tests { }) }); mock_search_service_2.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1851,7 +1855,7 @@ mod tests { #[tokio::test] async fn test_root_search_single_split_retry_single_node() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1872,11 +1876,11 @@ mod tests { let mut first_call = true; let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().times(2).returning( - move |_leaf_search_req: quickwit_proto::LeafSearchRequest| { + move |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { // requests from split 2 arrive here - simulate failure, then success if first_call { first_call = false; - Ok(quickwit_proto::LeafSearchResponse { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 0, partial_hits: Vec::new(), failed_splits: vec![SplitSearchError { @@ -1888,7 +1892,7 @@ mod tests { ..Default::default() }) } else { - Ok(quickwit_proto::LeafSearchResponse { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 1, partial_hits: vec![mock_partial_hit("split1", 2, 2)], failed_splits: Vec::new(), @@ -1899,8 +1903,8 @@ mod tests { }, ); mock_search_service.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -1923,7 +1927,7 @@ mod tests { #[tokio::test] async fn test_root_search_single_split_retry_single_node_fails() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -1944,8 +1948,8 @@ mod tests { let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().times(2).returning( - move |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + move |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 0, partial_hits: Vec::new(), failed_splits: vec![SplitSearchError { @@ -1959,7 +1963,7 @@ mod tests { }, ); mock_search_service.expect_fetch_docs().returning( - |_fetch_docs_req: quickwit_proto::FetchDocsRequest| { + |_fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { Err(SearchError::InternalError("mockerr docs".to_string())) }, ); @@ -1980,7 +1984,7 @@ mod tests { #[tokio::test] async fn test_root_search_one_splits_two_nodes_but_one_is_failing_for_split( ) -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -2001,9 +2005,9 @@ mod tests { // Service1 - broken node. let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( - move |_leaf_search_req: quickwit_proto::LeafSearchRequest| { + move |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { // retry requests from split 1 arrive here - Ok(quickwit_proto::LeafSearchResponse { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 1, partial_hits: vec![mock_partial_hit("split1", 2, 2)], failed_splits: Vec::new(), @@ -2013,8 +2017,8 @@ mod tests { }, ); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -2022,8 +2026,8 @@ mod tests { // Service2 - working node. let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( - move |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + move |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 0, partial_hits: Vec::new(), failed_splits: vec![SplitSearchError { @@ -2037,7 +2041,7 @@ mod tests { }, ); mock_search_service_2.expect_fetch_docs().returning( - |_fetch_docs_req: quickwit_proto::FetchDocsRequest| { + |_fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { Err(SearchError::InternalError("mockerr docs".to_string())) }, ); @@ -2063,7 +2067,7 @@ mod tests { #[tokio::test] async fn test_root_search_one_splits_two_nodes_but_one_is_failing_completely( ) -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -2085,8 +2089,8 @@ mod tests { // Service1 - working node. let mut mock_search_service_1 = MockSearchService::new(); mock_search_service_1.expect_leaf_search().returning( - move |_leaf_search_req: quickwit_proto::LeafSearchRequest| { - Ok(quickwit_proto::LeafSearchResponse { + move |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { + Ok(quickwit_proto::search::LeafSearchResponse { num_hits: 1, partial_hits: vec![mock_partial_hit("split1", 2, 2)], failed_splits: Vec::new(), @@ -2096,8 +2100,8 @@ mod tests { }, ); mock_search_service_1.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { - Ok(quickwit_proto::FetchDocsResponse { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -2105,12 +2109,12 @@ mod tests { // Service2 - broken node. let mut mock_search_service_2 = MockSearchService::new(); mock_search_service_2.expect_leaf_search().returning( - move |_leaf_search_req: quickwit_proto::LeafSearchRequest| { + move |_leaf_search_req: quickwit_proto::search::LeafSearchRequest| { Err(SearchError::InternalError("mockerr search".to_string())) }, ); mock_search_service_2.expect_fetch_docs().returning( - |_fetch_docs_req: quickwit_proto::FetchDocsRequest| { + |_fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { Err(SearchError::InternalError("mockerr docs".to_string())) }, ); @@ -2154,7 +2158,7 @@ mod tests { assert!(root_search( &SearcherContext::new(SearcherConfig::default()), - quickwit_proto::SearchRequest { + quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("invalid_field:\"test\"", &["body"]), max_hits: 10, @@ -2168,7 +2172,7 @@ mod tests { assert!(root_search( &SearcherContext::new(SearcherConfig::default()), - quickwit_proto::SearchRequest { + quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["invalid_field"]), max_hits: 10, @@ -2204,7 +2208,7 @@ mod tests { } }"#; - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -2244,7 +2248,7 @@ mod tests { #[tokio::test] async fn test_root_search_invalid_request() -> anyhow::Result<()> { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 10, @@ -2279,7 +2283,7 @@ mod tests { "Invalid argument: max value for start_offset is 10_000, but got 20000", ); - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: 20_000, @@ -2416,7 +2420,7 @@ mod tests { fn create_search_resp(hit_range: Range) -> LeafSearchResponse { let truncate_range = hit_range.start.min(TOTAL_NUM_HITS)..hit_range.end.min(TOTAL_NUM_HITS); - quickwit_proto::LeafSearchResponse { + quickwit_proto::search::LeafSearchResponse { num_hits: TOTAL_NUM_HITS as u64, partial_hits: truncate_range .map(|doc_id| mock_partial_hit("split1", u64::MAX - doc_id as u64, doc_id as u32)) @@ -2445,7 +2449,7 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1")])); let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_leaf_search().once().returning( - |req: quickwit_proto::LeafSearchRequest| { + |req: quickwit_proto::search::LeafSearchRequest| { let search_req: &SearchRequest = req.search_request.as_ref().unwrap(); // the leaf request does not need to know about the scroll_ttl. assert_eq!(search_req.start_offset, 0u64); @@ -2458,7 +2462,7 @@ mod tests { }, ); mock_search_service.expect_leaf_search().once().returning( - |req: quickwit_proto::LeafSearchRequest| { + |req: quickwit_proto::search::LeafSearchRequest| { let search_req: &SearchRequest = req.search_request.as_ref().unwrap(); // the leaf request does not need to know about the scroll_ttl. assert_eq!(search_req.start_offset, 0u64); @@ -2471,7 +2475,7 @@ mod tests { }, ); mock_search_service.expect_leaf_search().once().returning( - |req: quickwit_proto::LeafSearchRequest| { + |req: quickwit_proto::search::LeafSearchRequest| { let search_req: &SearchRequest = req.search_request.as_ref().unwrap(); // the leaf request does not need to know about the scroll_ttl. assert_eq!(search_req.start_offset, 0u64); @@ -2497,9 +2501,9 @@ mod tests { .expect_get_kv() .returning(move |get_kv_req| kv.read().unwrap().get(&get_kv_req.key).cloned()); mock_search_service.expect_fetch_docs().returning( - |fetch_docs_req: quickwit_proto::FetchDocsRequest| { + |fetch_docs_req: quickwit_proto::search::FetchDocsRequest| { assert!(fetch_docs_req.partial_hits.len() <= MAX_HITS_PER_PAGE); - Ok(quickwit_proto::FetchDocsResponse { + Ok(quickwit_proto::search::FetchDocsResponse { hits: get_doc_for_fetch_req(fetch_docs_req), }) }, @@ -2512,7 +2516,7 @@ mod tests { let mut count_seen_hits = 0; let mut scroll_id: String = { - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), max_hits: MAX_HITS_PER_PAGE as u64, diff --git a/quickwit/quickwit-search/src/scroll_context.rs b/quickwit/quickwit-search/src/scroll_context.rs index a846e42935a..8502f695f97 100644 --- a/quickwit/quickwit-search/src/scroll_context.rs +++ b/quickwit/quickwit-search/src/scroll_context.rs @@ -27,7 +27,7 @@ use base64::prelude::BASE64_STANDARD; use base64::Engine; use quickwit_common::uri::Uri; use quickwit_metastore::SplitMetadata; -use quickwit_proto::{LeafSearchResponse, PartialHit, SearchRequest}; +use quickwit_proto::search::{LeafSearchResponse, PartialHit, SearchRequest}; use serde::{Deserialize, Serialize}; use tokio::sync::RwLock; use ttl_cache::TtlCache; diff --git a/quickwit/quickwit-search/src/search_response_rest.rs b/quickwit/quickwit-search/src/search_response_rest.rs index eeddbbd00df..20212ec9dbb 100644 --- a/quickwit/quickwit-search/src/search_response_rest.rs +++ b/quickwit/quickwit-search/src/search_response_rest.rs @@ -20,7 +20,7 @@ use std::convert::TryFrom; use quickwit_common::truncate_str; -use quickwit_proto::SearchResponse; +use quickwit_proto::search::SearchResponse; use serde::{Deserialize, Serialize}; use serde_json::Value as JsonValue; diff --git a/quickwit/quickwit-search/src/search_stream/leaf.rs b/quickwit/quickwit-search/src/search_stream/leaf.rs index 6ef89827731..89046a161e2 100644 --- a/quickwit/quickwit-search/src/search_stream/leaf.rs +++ b/quickwit/quickwit-search/src/search_stream/leaf.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use futures::{FutureExt, StreamExt}; use quickwit_doc_mapper::DocMapper; -use quickwit_proto::{ +use quickwit_proto::search::{ LeafSearchStreamResponse, OutputFormat, SearchRequest, SearchStreamRequest, SplitIdAndFooterOffsets, }; diff --git a/quickwit/quickwit-search/src/search_stream/mod.rs b/quickwit/quickwit-search/src/search_stream/mod.rs index 0529b5e9354..9fa65200a21 100644 --- a/quickwit/quickwit-search/src/search_stream/mod.rs +++ b/quickwit/quickwit-search/src/search_stream/mod.rs @@ -27,7 +27,7 @@ use std::io::Write; pub use collector::FastFieldCollector; pub use leaf::leaf_search_stream; -use quickwit_proto::OutputFormat; +use quickwit_proto::search::OutputFormat; pub use root::root_search_stream; use tantivy::columnar::MonotonicallyMappableToU64; diff --git a/quickwit/quickwit-search/src/search_stream/root.rs b/quickwit/quickwit-search/src/search_stream/root.rs index 64ebef9ad0f..d4754c44258 100644 --- a/quickwit/quickwit-search/src/search_stream/root.rs +++ b/quickwit/quickwit-search/src/search_stream/root.rs @@ -24,7 +24,7 @@ use futures::{StreamExt, TryStreamExt}; use quickwit_common::uri::Uri; use quickwit_config::build_doc_mapper; use quickwit_metastore::Metastore; -use quickwit_proto::{LeafSearchStreamRequest, SearchRequest, SearchStreamRequest}; +use quickwit_proto::search::{LeafSearchStreamRequest, SearchRequest, SearchStreamRequest}; use quickwit_query::query_ast::QueryAst; use tokio_stream::StreamMap; use tracing::*; @@ -122,7 +122,7 @@ mod tests { use quickwit_indexing::mock_split; use quickwit_metastore::{IndexMetadata, MockMetastore}; - use quickwit_proto::OutputFormat; + use quickwit_proto::search::OutputFormat; use quickwit_query::query_ast::qast_helper; use tokio_stream::wrappers::UnboundedReceiverStream; @@ -131,7 +131,7 @@ mod tests { #[tokio::test] async fn test_root_search_stream_single_split() -> anyhow::Result<()> { - let request = quickwit_proto::SearchStreamRequest { + let request = quickwit_proto::search::SearchStreamRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), fast_field: "timestamp".to_string(), @@ -152,16 +152,16 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1")])); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); - result_sender.send(Ok(quickwit_proto::LeafSearchStreamResponse { + result_sender.send(Ok(quickwit_proto::search::LeafSearchStreamResponse { data: b"123".to_vec(), split_id: "split_1".to_string(), }))?; - result_sender.send(Ok(quickwit_proto::LeafSearchStreamResponse { + result_sender.send(Ok(quickwit_proto::search::LeafSearchStreamResponse { data: b"456".to_vec(), split_id: "split_1".to_string(), }))?; mock_search_service.expect_leaf_search_stream().return_once( - |_leaf_search_req: quickwit_proto::LeafSearchStreamRequest| { + |_leaf_search_req: quickwit_proto::search::LeafSearchStreamRequest| { Ok(UnboundedReceiverStream::new(result_receiver)) }, ); @@ -183,7 +183,7 @@ mod tests { #[tokio::test] async fn test_root_search_stream_single_split_partitionned() -> anyhow::Result<()> { - let request = quickwit_proto::SearchStreamRequest { + let request = quickwit_proto::search::SearchStreamRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), fast_field: "timestamp".to_string(), @@ -205,16 +205,16 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1")])); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); - result_sender.send(Ok(quickwit_proto::LeafSearchStreamResponse { + result_sender.send(Ok(quickwit_proto::search::LeafSearchStreamResponse { data: b"123".to_vec(), split_id: "1".to_string(), }))?; - result_sender.send(Ok(quickwit_proto::LeafSearchStreamResponse { + result_sender.send(Ok(quickwit_proto::search::LeafSearchStreamResponse { data: b"456".to_vec(), split_id: "2".to_string(), }))?; mock_search_service.expect_leaf_search_stream().return_once( - |_leaf_search_req: quickwit_proto::LeafSearchStreamRequest| { + |_leaf_search_req: quickwit_proto::search::LeafSearchStreamRequest| { Ok(UnboundedReceiverStream::new(result_receiver)) }, ); @@ -234,7 +234,7 @@ mod tests { #[tokio::test] async fn test_root_search_stream_single_split_with_error() -> anyhow::Result<()> { - let request = quickwit_proto::SearchStreamRequest { + let request = quickwit_proto::search::SearchStreamRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), fast_field: "timestamp".to_string(), @@ -255,7 +255,7 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split1"), mock_split("split2")])); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); - result_sender.send(Ok(quickwit_proto::LeafSearchStreamResponse { + result_sender.send(Ok(quickwit_proto::search::LeafSearchStreamResponse { data: b"123".to_vec(), split_id: "split1".to_string(), }))?; @@ -264,7 +264,7 @@ mod tests { .expect_leaf_search_stream() .withf(|request| request.split_offsets.len() == 2) // First request. .return_once( - |_leaf_search_req: quickwit_proto::LeafSearchStreamRequest| { + |_leaf_search_req: quickwit_proto::search::LeafSearchStreamRequest| { Ok(UnboundedReceiverStream::new(result_receiver)) }, ); @@ -272,7 +272,7 @@ mod tests { .expect_leaf_search_stream() .withf(|request| request.split_offsets.len() == 1) // Retry request on the failed split. .return_once( - |_leaf_search_req: quickwit_proto::LeafSearchStreamRequest| { + |_leaf_search_req: quickwit_proto::search::LeafSearchStreamRequest| { Err(SearchError::InternalError("error".to_string())) }, ); @@ -308,7 +308,7 @@ mod tests { let search_job_placer = SearchJobPlacer::new(searcher_pool); assert!(root_search_stream( - quickwit_proto::SearchStreamRequest { + quickwit_proto::search::SearchStreamRequest { index_id: "test-index".to_string(), query_ast: qast_helper(r#"invalid_field:"test""#, &[]), fast_field: "timestamp".to_string(), @@ -323,7 +323,7 @@ mod tests { .is_err()); assert!(root_search_stream( - quickwit_proto::SearchStreamRequest { + quickwit_proto::search::SearchStreamRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["invalid_field"]), fast_field: "timestamp".to_string(), diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index 01d18374aaa..ecb1ad497b7 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -28,7 +28,7 @@ use quickwit_common::uri::Uri; use quickwit_config::SearcherConfig; use quickwit_doc_mapper::DocMapper; use quickwit_metastore::Metastore; -use quickwit_proto::{ +use quickwit_proto::search::{ FetchDocsRequest, FetchDocsResponse, GetKvRequest, Hit, LeafListTermsRequest, LeafListTermsResponse, LeafSearchRequest, LeafSearchResponse, LeafSearchStreamRequest, LeafSearchStreamResponse, ListTermsRequest, ListTermsResponse, PutKvRequest, ScrollRequest, diff --git a/quickwit/quickwit-search/src/tests.rs b/quickwit/quickwit-search/src/tests.rs index 7902db69fa2..0c5672a7836 100644 --- a/quickwit/quickwit-search/src/tests.rs +++ b/quickwit/quickwit-search/src/tests.rs @@ -24,8 +24,9 @@ use quickwit_config::SearcherConfig; use quickwit_doc_mapper::DefaultDocMapper; use quickwit_indexing::TestSandbox; use quickwit_opentelemetry::otlp::TraceId; -use quickwit_proto::{ - LeafListTermsResponse, SearchRequest, SortByValue, SortField, SortOrder, SortValue, +use quickwit_proto::search::{ + LeafListTermsResponse, ListTermsRequest, SearchRequest, SortByValue, SortField, SortOrder, + SortValue, }; use quickwit_query::query_ast::{qast_helper, query_ast_from_user_text}; use serde_json::{json, Value as JsonValue}; @@ -976,7 +977,7 @@ async fn test_search_util(test_sandbox: &TestSandbox, query: &str) -> Vec { .into_iter() .map(|split_meta| extract_split_and_footer_offsets(&split_meta.split_metadata)) .collect(); - let request = quickwit_proto::SearchRequest { + let request = SearchRequest { index_id: test_sandbox.index_uid().index_id().to_string(), query_ast: qast_helper(query, &[]), max_hits: 100, @@ -1610,7 +1611,7 @@ async fn test_single_node_list_terms() -> anyhow::Result<()> { let searcher_context = Arc::new(SearcherContext::new(SearcherConfig::default())); { - let request = quickwit_proto::ListTermsRequest { + let request = ListTermsRequest { index_id: test_sandbox.index_uid().index_id().to_string(), field: "title".to_string(), start_key: None, @@ -1631,7 +1632,7 @@ async fn test_single_node_list_terms() -> anyhow::Result<()> { assert_eq!(terms, &["beagle", "snoopy",]); } { - let request = quickwit_proto::ListTermsRequest { + let request = ListTermsRequest { index_id: test_sandbox.index_uid().index_id().to_string(), field: "title".to_string(), start_key: None, @@ -1652,7 +1653,7 @@ async fn test_single_node_list_terms() -> anyhow::Result<()> { assert_eq!(terms, &["beagle"]); } { - let request = quickwit_proto::ListTermsRequest { + let request = ListTermsRequest { index_id: test_sandbox.index_uid().index_id().to_string(), field: "title".to_string(), start_key: Some("casper".as_bytes().to_vec()), @@ -1673,7 +1674,7 @@ async fn test_single_node_list_terms() -> anyhow::Result<()> { assert_eq!(terms, &["snoopy"]); } { - let request = quickwit_proto::ListTermsRequest { + let request = ListTermsRequest { index_id: test_sandbox.index_uid().index_id().to_string(), field: "title".to_string(), start_key: None, diff --git a/quickwit/quickwit-serve/src/delete_task_api/handler.rs b/quickwit/quickwit-serve/src/delete_task_api/handler.rs index 167c4687280..8b34f5a8960 100644 --- a/quickwit/quickwit-serve/src/delete_task_api/handler.rs +++ b/quickwit/quickwit-serve/src/delete_task_api/handler.rs @@ -23,7 +23,8 @@ use quickwit_config::build_doc_mapper; use quickwit_janitor::error::JanitorError; use quickwit_metastore::{Metastore, MetastoreError}; use quickwit_proto::metastore::{DeleteQuery, DeleteTask}; -use quickwit_proto::{IndexUid, SearchRequest}; +use quickwit_proto::search::SearchRequest; +use quickwit_proto::IndexUid; use quickwit_query::query_ast::{query_ast_from_user_text, QueryAst}; use serde::Deserialize; use warp::{Filter, Rejection}; diff --git a/quickwit/quickwit-serve/src/elastic_search_api/mod.rs b/quickwit/quickwit-serve/src/elastic_search_api/mod.rs index ad51542e2b1..6f34f55ff75 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/mod.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/mod.rs @@ -117,7 +117,7 @@ mod tests { mock_search_service .expect_root_search() .with(predicate::function( - |search_request: &quickwit_proto::SearchRequest| { + |search_request: &quickwit_proto::search::SearchRequest| { (search_request.index_id == "index-1" && search_request.start_offset == 5 && search_request.max_hits == 20) diff --git a/quickwit/quickwit-serve/src/elastic_search_api/model/mod.rs b/quickwit/quickwit-serve/src/elastic_search_api/model/mod.rs index 9a64c3e9358..0e7fe8514e4 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/model/mod.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/model/mod.rs @@ -31,7 +31,7 @@ pub use error::ElasticSearchError; pub use multi_search::{ MultiSearchHeader, MultiSearchQueryParams, MultiSearchResponse, MultiSearchSingleResponse, }; -use quickwit_proto::SortOrder; +use quickwit_proto::search::SortOrder; pub use scroll::ScrollQueryParams; pub use search_body::SearchBody; pub use search_query_params::SearchQueryParams; diff --git a/quickwit/quickwit-serve/src/elastic_search_api/model/search_body.rs b/quickwit/quickwit-serve/src/elastic_search_api/model/search_body.rs index 7abb689464a..060cccdb9e0 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/model/search_body.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/model/search_body.rs @@ -20,7 +20,7 @@ use std::collections::BTreeSet; use std::fmt; -use quickwit_proto::SortOrder; +use quickwit_proto::search::SortOrder; use quickwit_query::{ElasticQueryDsl, OneFieldMap}; use serde::de::{MapAccess, Visitor}; use serde::{Deserialize, Deserializer, Serialize}; diff --git a/quickwit/quickwit-serve/src/elastic_search_api/model/search_query_params.rs b/quickwit/quickwit-serve/src/elastic_search_api/model/search_query_params.rs index c108f055fe1..d164aba2b32 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/model/search_query_params.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/model/search_query_params.rs @@ -20,7 +20,7 @@ use std::str::FromStr; use std::time::Duration; -use quickwit_proto::SortOrder; +use quickwit_proto::search::SortOrder; use quickwit_query::BooleanOperand; use quickwit_search::SearchError; use serde::{Deserialize, Serialize}; diff --git a/quickwit/quickwit-serve/src/elastic_search_api/rest_handler.rs b/quickwit/quickwit-serve/src/elastic_search_api/rest_handler.rs index bec9ba5e5cf..e61c1825ea1 100644 --- a/quickwit/quickwit-serve/src/elastic_search_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/elastic_search_api/rest_handler.rs @@ -30,7 +30,8 @@ use hyper::StatusCode; use itertools::Itertools; use quickwit_common::truncate_str; use quickwit_config::NodeConfig; -use quickwit_proto::{ScrollRequest, SearchResponse, ServiceErrorCode}; +use quickwit_proto::search::{ScrollRequest, SearchResponse}; +use quickwit_proto::ServiceErrorCode; use quickwit_query::query_ast::{QueryAst, UserInputQuery}; use quickwit_query::BooleanOperand; use quickwit_search::{SearchError, SearchService}; @@ -129,7 +130,7 @@ fn build_request_for_es_api( index_id: String, search_params: SearchQueryParams, search_body: SearchBody, -) -> Result { +) -> Result { let default_operator = search_params.default_operator.unwrap_or(BooleanOperand::Or); // The query string, if present, takes priority over what can be in the request // body. @@ -156,12 +157,12 @@ fn build_request_for_es_api( let max_hits = search_params.size.or(search_body.size).unwrap_or(10); let start_offset = search_params.from.or(search_body.from).unwrap_or(0); - let sort_fields: Vec = search_params + let sort_fields: Vec = search_params .sort_fields()? .or_else(|| search_body.sort.clone()) .unwrap_or_default() .iter() - .map(|sort_field| quickwit_proto::SortField { + .map(|sort_field| quickwit_proto::search::SortField { field_name: sort_field.field.to_string(), sort_order: sort_field.order as i32, }) @@ -175,7 +176,7 @@ fn build_request_for_es_api( let scroll_duration: Option = search_params.parse_scroll_ttl()?; let scroll_ttl_secs: Option = scroll_duration.map(|duration| duration.as_secs() as u32); - Ok(quickwit_proto::SearchRequest { + Ok(quickwit_proto::search::SearchRequest { index_id, query_ast: serde_json::to_string(&query_ast).expect("Failed to serialize QueryAst"), max_hits, @@ -205,7 +206,7 @@ async fn es_compat_index_search( Ok(search_response_rest) } -fn convert_hit(hit: quickwit_proto::Hit) -> ElasticHit { +fn convert_hit(hit: quickwit_proto::search::Hit) -> ElasticHit { let fields: BTreeMap = serde_json::from_str(&hit.json).unwrap_or_default(); ElasticHit { diff --git a/quickwit/quickwit-serve/src/grpc.rs b/quickwit/quickwit-serve/src/grpc.rs index 8da4d2bdea4..b8045018384 100644 --- a/quickwit/quickwit-serve/src/grpc.rs +++ b/quickwit/quickwit-serve/src/grpc.rs @@ -33,10 +33,10 @@ use quickwit_proto::control_plane::ControlPlaneServiceGrpcServerAdapter; use quickwit_proto::indexing::indexing_service_grpc_server::IndexingServiceGrpcServer; use quickwit_proto::indexing::{IndexingServiceClient, IndexingServiceGrpcServerAdapter}; use quickwit_proto::jaeger::storage::v1::span_reader_plugin_server::SpanReaderPluginServer; -use quickwit_proto::metastore::metastore_service_server::MetastoreServiceServer; +use quickwit_proto::metastore::MetastoreServiceServer; use quickwit_proto::opentelemetry::proto::collector::logs::v1::logs_service_server::LogsServiceServer; use quickwit_proto::opentelemetry::proto::collector::trace::v1::trace_service_server::TraceServiceServer; -use quickwit_proto::search_service_server::SearchServiceServer; +use quickwit_proto::search::search_service_server::SearchServiceServer; use quickwit_proto::tonic::codegen::CompressionEncoding; use quickwit_proto::tonic::transport::Server; use tracing::*; diff --git a/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs b/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs index fc2f609bdae..5c792568545 100644 --- a/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs +++ b/quickwit/quickwit-serve/src/search_api/grpc_adapter.rs @@ -21,10 +21,12 @@ use std::sync::Arc; use async_trait::async_trait; use futures::TryStreamExt; +use quickwit_proto::search::{ + search_service_server as grpc, GetKvRequest, GetKvResponse, LeafSearchStreamRequest, + LeafSearchStreamResponse, +}; use quickwit_proto::{ - convert_to_grpc_result, search_service_server as grpc, set_parent_span_from_request_metadata, - tonic, GetKvRequest, GetKvResponse, LeafSearchStreamRequest, LeafSearchStreamResponse, - ServiceError, + convert_to_grpc_result, set_parent_span_from_request_metadata, tonic, ServiceError, }; use quickwit_search::SearchService; use tracing::instrument; @@ -43,8 +45,8 @@ impl grpc::SearchService for GrpcSearchAdapter { #[instrument(skip(self, request))] async fn root_search( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { set_parent_span_from_request_metadata(request.metadata()); let search_request = request.into_inner(); let search_result = self.0.root_search(search_request).await; @@ -54,8 +56,8 @@ impl grpc::SearchService for GrpcSearchAdapter { #[instrument(skip(self, request))] async fn leaf_search( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { set_parent_span_from_request_metadata(request.metadata()); let leaf_search_request = request.into_inner(); let leaf_search_result = self.0.leaf_search(leaf_search_request).await; @@ -65,8 +67,8 @@ impl grpc::SearchService for GrpcSearchAdapter { #[instrument(skip(self, request))] async fn fetch_docs( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { set_parent_span_from_request_metadata(request.metadata()); let fetch_docs_request = request.into_inner(); let fetch_docs_result = self.0.fetch_docs(fetch_docs_request).await; @@ -99,8 +101,8 @@ impl grpc::SearchService for GrpcSearchAdapter { #[instrument(skip(self, request))] async fn root_list_terms( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { set_parent_span_from_request_metadata(request.metadata()); let search_request = request.into_inner(); let search_result = self.0.root_list_terms(search_request).await; @@ -110,8 +112,8 @@ impl grpc::SearchService for GrpcSearchAdapter { #[instrument(skip(self, request))] async fn leaf_list_terms( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { set_parent_span_from_request_metadata(request.metadata()); let leaf_search_request = request.into_inner(); let leaf_search_result = self.0.leaf_list_terms(leaf_search_request).await; @@ -120,8 +122,8 @@ impl grpc::SearchService for GrpcSearchAdapter { async fn scroll( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { let scroll_request = request.into_inner(); let scroll_result = self.0.scroll(scroll_request).await; convert_to_grpc_result(scroll_result) @@ -130,12 +132,14 @@ impl grpc::SearchService for GrpcSearchAdapter { #[instrument(skip(self, request))] async fn put_kv( &self, - request: tonic::Request, - ) -> Result, tonic::Status> { + request: tonic::Request, + ) -> Result, tonic::Status> { set_parent_span_from_request_metadata(request.metadata()); let put_request = request.into_inner(); self.0.put_kv(put_request).await; - Ok(tonic::Response::new(quickwit_proto::PutKvResponse {})) + Ok(tonic::Response::new( + quickwit_proto::search::PutKvResponse {}, + )) } #[instrument(skip(self, request))] diff --git a/quickwit/quickwit-serve/src/search_api/mod.rs b/quickwit/quickwit-serve/src/search_api/mod.rs index fe69f94ebeb..2eba31c6a08 100644 --- a/quickwit/quickwit-serve/src/search_api/mod.rs +++ b/quickwit/quickwit-serve/src/search_api/mod.rs @@ -34,8 +34,9 @@ mod tests { use futures::TryStreamExt; use quickwit_indexing::mock_split; use quickwit_metastore::{IndexMetadata, MockMetastore}; - use quickwit_proto::search_service_server::SearchServiceServer; - use quickwit_proto::{tonic, OutputFormat}; + use quickwit_proto::search::search_service_server::SearchServiceServer; + use quickwit_proto::search::OutputFormat; + use quickwit_proto::tonic; use quickwit_query::query_ast::qast_helper; use quickwit_search::{ create_search_client_from_grpc_addr, root_search_stream, ClusterClient, MockSearchService, @@ -64,7 +65,7 @@ mod tests { #[tokio::test] async fn test_serve_search_stream_with_a_leaf_error_on_leaf_node() -> anyhow::Result<()> { // This test aims at checking the client gRPC implementation. - let request = quickwit_proto::SearchStreamRequest { + let request = quickwit_proto::search::SearchStreamRequest { index_id: "test-index".to_string(), query_ast: qast_helper("test", &["body"]), snippet_fields: Vec::new(), @@ -88,7 +89,7 @@ mod tests { .returning(|_filter| Ok(vec![mock_split("split_1"), mock_split("split_2")])); let mut mock_search_service = MockSearchService::new(); let (result_sender, result_receiver) = tokio::sync::mpsc::unbounded_channel(); - result_sender.send(Ok(quickwit_proto::LeafSearchStreamResponse { + result_sender.send(Ok(quickwit_proto::search::LeafSearchStreamResponse { data: b"123".to_vec(), split_id: "split_1".to_string(), }))?; @@ -99,7 +100,7 @@ mod tests { .expect_leaf_search_stream() .withf(|request| request.split_offsets.len() == 2) // First request. .return_once( - |_leaf_search_req: quickwit_proto::LeafSearchStreamRequest| { + |_leaf_search_req: quickwit_proto::search::LeafSearchStreamRequest| { Ok(UnboundedReceiverStream::new(result_receiver)) }, ); @@ -107,7 +108,7 @@ mod tests { .expect_leaf_search_stream() .withf(|request| request.split_offsets.len() == 1) // Retry request on the failing split. .return_once( - |_leaf_search_req: quickwit_proto::LeafSearchStreamRequest| { + |_leaf_search_req: quickwit_proto::search::LeafSearchStreamRequest| { Err(SearchError::InternalError( "Error again on `split2`".to_string(), )) diff --git a/quickwit/quickwit-serve/src/search_api/rest_handler.rs b/quickwit/quickwit-serve/src/search_api/rest_handler.rs index 86a74657e25..71ba0d8c286 100644 --- a/quickwit/quickwit-serve/src/search_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/search_api/rest_handler.rs @@ -23,7 +23,8 @@ use std::sync::Arc; use futures::stream::StreamExt; use hyper::header::HeaderValue; use hyper::HeaderMap; -use quickwit_proto::{OutputFormat, ServiceError, SortField, SortOrder}; +use quickwit_proto::search::{OutputFormat, SortField, SortOrder}; +use quickwit_proto::ServiceError; use quickwit_query::query_ast::query_ast_from_user_text; use quickwit_search::{SearchError, SearchResponseRest, SearchService}; use serde::{de, Deserialize, Deserializer, Serialize, Serializer}; @@ -202,13 +203,13 @@ pub struct SearchRequestQueryString { pub fn search_request_from_api_request( index_id: String, search_request: SearchRequestQueryString, -) -> Result { +) -> Result { // The query ast below may still contain user input query. The actual // parsing of the user query will happen in the root service, and might require // the user of the docmapper default fields (which we do not have at this point). let query_ast = query_ast_from_user_text(&search_request.query, search_request.search_fields); let query_ast_json = serde_json::to_string(&query_ast)?; - let search_request = quickwit_proto::SearchRequest { + let search_request = quickwit_proto::search::SearchRequest { index_id, query_ast: query_ast_json, snippet_fields: search_request.snippet_fields.unwrap_or_default(), @@ -371,7 +372,7 @@ async fn search_stream_endpoint( ) -> Result { let query_ast = query_ast_from_user_text(&search_request.query, search_request.search_fields); let query_ast_json = serde_json::to_string(&query_ast)?; - let request = quickwit_proto::SearchStreamRequest { + let request = quickwit_proto::search::SearchStreamRequest { index_id, query_ast: query_ast_json, snippet_fields: search_request.snippet_fields.unwrap_or_default(), @@ -753,7 +754,7 @@ mod tests { async fn test_rest_search_api_route_serialize_with_results() -> anyhow::Result<()> { let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_root_search().returning(|_| { - Ok(quickwit_proto::SearchResponse { + Ok(quickwit_proto::search::SearchResponse { hits: Vec::new(), num_hits: 10, elapsed_time_micros: 16, @@ -783,7 +784,7 @@ mod tests { mock_search_service .expect_root_search() .with(predicate::function( - |search_request: &quickwit_proto::SearchRequest| { + |search_request: &quickwit_proto::search::SearchRequest| { search_request.start_offset == 5 && search_request.max_hits == 30 }, )) @@ -966,8 +967,8 @@ mod tests { async fn test_rest_search_api_route_serialize_results_with_snippet() -> anyhow::Result<()> { let mut mock_search_service = MockSearchService::new(); mock_search_service.expect_root_search().returning(|_| { - Ok(quickwit_proto::SearchResponse { - hits: vec![quickwit_proto::Hit { + Ok(quickwit_proto::search::SearchResponse { + hits: vec![quickwit_proto::search::Hit { json: r#"{"title": "foo", "body": "foo bar baz"}"#.to_string(), partial_hit: None, snippet: Some(r#"{"title": [], "body": ["foo bar baz"]}"#.to_string()),