diff --git a/Makefile b/Makefile index f86b299567..4617ff44b6 100644 --- a/Makefile +++ b/Makefile @@ -27,7 +27,7 @@ docker-compose-up: COMPOSE_PROFILES=$(DOCKER_SERVICES) docker compose -f docker-compose.yml up -d --remove-orphans --wait docker-compose-down: - docker compose -f docker-compose.yml down --remove-orphans + docker compose -p quickwit down --remove-orphans docker-compose-logs: docker compose logs -f -t diff --git a/docker-compose.yml b/docker-compose.yml index 4fdf9dc290..2cfeb7580c 100644 --- a/docker-compose.yml +++ b/docker-compose.yml @@ -15,6 +15,8 @@ # first if they are already tagged latest and volumes if their content is # incompatible with the latest version, as in case of postgres. +name: quickwit + networks: default: name: quickwit-network diff --git a/quickwit/quickwit-cli/src/index/mod.rs b/quickwit/quickwit-cli/src/index/mod.rs index d355191812..efc253b966 100644 --- a/quickwit/quickwit-cli/src/index/mod.rs +++ b/quickwit/quickwit-cli/src/index/mod.rs @@ -44,6 +44,7 @@ use quickwit_indexing::models::IndexingStatistics; use quickwit_indexing::IndexingPipeline; use quickwit_metastore::{IndexMetadata, Split, SplitState}; use quickwit_proto::search::{CountHits, SortField, SortOrder}; +use quickwit_proto::types::IndexId; use quickwit_rest_client::models::IngestSource; use quickwit_rest_client::rest_client::{CommitType, IngestEvent}; use quickwit_search::SearchResponseRest; @@ -200,7 +201,7 @@ pub fn build_index_command() -> Command { #[derive(Debug, Eq, PartialEq)] pub struct ClearIndexArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub assume_yes: bool, } @@ -215,13 +216,13 @@ pub struct CreateIndexArgs { #[derive(Debug, Eq, PartialEq)] pub struct DescribeIndexArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, } #[derive(Debug, Eq, PartialEq)] pub struct IngestDocsArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub input_path_opt: Option, pub batch_size_limit_opt: Option, pub commit_type: CommitType, @@ -230,7 +231,7 @@ pub struct IngestDocsArgs { #[derive(Debug, Eq, PartialEq)] pub struct SearchIndexArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub query: String, pub aggregation: Option, pub max_hits: usize, @@ -245,7 +246,7 @@ pub struct SearchIndexArgs { #[derive(Debug, Eq, PartialEq)] pub struct DeleteIndexArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub dry_run: bool, pub assume_yes: bool, } @@ -528,7 +529,7 @@ where I: IntoIterator { #[derive(Tabled)] struct IndexRow { #[tabled(rename = "Index ID")] - index_id: String, + index_id: IndexId, #[tabled(rename = "Index URI")] index_uri: Uri, } @@ -548,7 +549,7 @@ pub async fn describe_index_cli(args: DescribeIndexArgs) -> anyhow::Result<()> { } pub struct IndexStats { - pub index_id: String, + pub index_id: IndexId, pub index_uri: Uri, pub num_published_splits: usize, pub size_published_splits: ByteSize, diff --git a/quickwit/quickwit-cli/src/index/update.rs b/quickwit/quickwit-cli/src/index/update.rs index c79f30357e..f417dbb021 100644 --- a/quickwit/quickwit-cli/src/index/update.rs +++ b/quickwit/quickwit-cli/src/index/update.rs @@ -21,6 +21,7 @@ use anyhow::{bail, Context}; use clap::{arg, ArgMatches, Command}; use colored::Colorize; use quickwit_config::{RetentionPolicy, SearchSettings}; +use quickwit_proto::types::IndexId; use quickwit_serve::IndexUpdates; use tracing::debug; @@ -65,7 +66,7 @@ pub fn build_index_update_command() -> Command { #[derive(Debug, Eq, PartialEq)] pub struct RetentionPolicyArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub disable: bool, pub period: Option, pub schedule: Option, @@ -74,7 +75,7 @@ pub struct RetentionPolicyArgs { #[derive(Debug, Eq, PartialEq)] pub struct SearchSettingsArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub default_search_fields: Vec, } diff --git a/quickwit/quickwit-cli/src/source.rs b/quickwit/quickwit-cli/src/source.rs index c61820c9b1..1a1948fdd9 100644 --- a/quickwit/quickwit-cli/src/source.rs +++ b/quickwit/quickwit-cli/src/source.rs @@ -26,6 +26,7 @@ use itertools::Itertools; use quickwit_common::uri::Uri; use quickwit_config::{validate_identifier, ConfigFormat, SourceConfig}; use quickwit_metastore::checkpoint::SourceCheckpoint; +use quickwit_proto::types::{IndexId, SourceId}; use quickwit_storage::{load_file, StorageResolver}; use serde_json::Value as JsonValue; use tabled::{Table, Tabled}; @@ -142,44 +143,44 @@ pub fn build_source_command() -> Command { #[derive(Debug, Eq, PartialEq)] pub struct CreateSourceArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub source_config_uri: Uri, } #[derive(Debug, Eq, PartialEq)] pub struct ToggleSourceArgs { pub client_args: ClientArgs, - pub index_id: String, - pub source_id: String, + pub index_id: IndexId, + pub source_id: SourceId, pub enable: bool, } #[derive(Debug, Eq, PartialEq)] pub struct DeleteSourceArgs { pub client_args: ClientArgs, - pub index_id: String, - pub source_id: String, + pub index_id: IndexId, + pub source_id: SourceId, pub assume_yes: bool, } #[derive(Debug, Eq, PartialEq)] pub struct DescribeSourceArgs { pub client_args: ClientArgs, - pub index_id: String, - pub source_id: String, + pub index_id: IndexId, + pub source_id: SourceId, } #[derive(Debug, Eq, PartialEq)] pub struct ListSourcesArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, } #[derive(Debug, Eq, PartialEq)] pub struct ResetCheckpointArgs { pub client_args: ClientArgs, - pub index_id: String, - pub source_id: String, + pub index_id: IndexId, + pub source_id: SourceId, pub assume_yes: bool, } @@ -469,7 +470,7 @@ where I: IntoIterator { #[derive(Tabled)] struct SourceRow { #[tabled(rename = "ID")] - source_id: String, + source_id: SourceId, #[tabled(rename = "Type")] source_type: String, #[tabled(rename = "Enabled")] diff --git a/quickwit/quickwit-cli/src/split.rs b/quickwit/quickwit-cli/src/split.rs index dbde45902a..97035ff11e 100644 --- a/quickwit/quickwit-cli/src/split.rs +++ b/quickwit/quickwit-cli/src/split.rs @@ -24,6 +24,7 @@ use clap::{arg, ArgMatches, Command}; use colored::Colorize; use itertools::Itertools; use quickwit_metastore::{Split, SplitState}; +use quickwit_proto::types::{IndexId, SplitId}; use quickwit_serve::ListSplitsQueryParams; use tabled::{Table, Tabled}; use time::{format_description, Date, OffsetDateTime, PrimitiveDateTime}; @@ -133,7 +134,7 @@ impl FromStr for OutputFormat { #[derive(Debug, PartialEq)] pub struct ListSplitArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub offset: Option, pub limit: Option, pub split_states: Option>, @@ -147,7 +148,7 @@ pub struct ListSplitArgs { #[derive(Debug, Eq, PartialEq)] pub struct MarkForDeletionArgs { pub client_args: ClientArgs, - pub index_id: String, + pub index_id: IndexId, pub split_ids: Vec, pub assume_yes: bool, } @@ -155,8 +156,8 @@ pub struct MarkForDeletionArgs { #[derive(Debug, Eq, PartialEq)] pub struct DescribeSplitArgs { pub client_args: ClientArgs, - pub index_id: String, - pub split_id: String, + pub index_id: IndexId, + pub split_id: SplitId, pub verbose: bool, } @@ -470,7 +471,7 @@ fn parse_split_state(split_state_arg: &str) -> anyhow::Result { #[derive(Tabled)] struct SplitRow { #[tabled(rename = "ID")] - split_id: String, + split_id: SplitId, #[tabled(rename = "State")] split_state: SplitState, #[tabled(rename = "Num docs")] diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 4ccc20f163..2d894a7eb0 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -40,9 +40,7 @@ use quickwit_config::{ VecSourceParams, CLI_SOURCE_ID, }; use quickwit_index_management::{clear_cache_directory, IndexService}; -use quickwit_indexing::actors::{ - IndexingService, MergePipeline, MergePipelineId, MergeSchedulerService, -}; +use quickwit_indexing::actors::{IndexingService, MergePipeline, MergeSchedulerService}; use quickwit_indexing::models::{ DetachIndexingPipeline, DetachMergePipeline, IndexingStatistics, SpawnPipeline, }; @@ -52,7 +50,7 @@ use quickwit_metastore::IndexMetadataResponseExt; use quickwit_proto::indexing::CpuCapacity; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; use quickwit_proto::search::{CountHits, SearchResponse}; -use quickwit_proto::types::{NodeId, PipelineUid}; +use quickwit_proto::types::{IndexId, PipelineUid, SourceId, SplitId}; use quickwit_search::{single_node_search, SearchResponseRest}; use quickwit_serve::{ search_request_from_api_request, BodyFormat, SearchRequestQueryString, SortBy, @@ -174,7 +172,7 @@ pub fn build_tool_command() -> Command { #[derive(Debug, Eq, PartialEq)] pub struct LocalIngestDocsArgs { pub config_uri: Uri, - pub index_id: String, + pub index_id: IndexId, pub input_path_opt: Option, pub input_format: SourceInputFormat, pub overwrite: bool, @@ -185,7 +183,7 @@ pub struct LocalIngestDocsArgs { #[derive(Debug, Eq, PartialEq)] pub struct LocalSearchArgs { pub config_uri: Uri, - pub index_id: String, + pub index_id: IndexId, pub query: String, pub aggregation: Option, pub max_hits: usize, @@ -200,7 +198,7 @@ pub struct LocalSearchArgs { #[derive(Debug, Eq, PartialEq)] pub struct GarbageCollectIndexArgs { pub config_uri: Uri, - pub index_id: String, + pub index_id: IndexId, pub grace_period: Duration, pub dry_run: bool, } @@ -208,15 +206,15 @@ pub struct GarbageCollectIndexArgs { #[derive(Debug, Eq, PartialEq)] pub struct MergeArgs { pub config_uri: Uri, - pub index_id: String, - pub source_id: String, + pub index_id: IndexId, + pub source_id: SourceId, } #[derive(Debug, Eq, PartialEq)] pub struct ExtractSplitArgs { pub config_uri: Uri, - pub index_id: String, - pub split_id: String, + pub index_id: IndexId, + pub split_id: SplitId, pub target_dir: PathBuf, } @@ -479,7 +477,7 @@ pub async fn local_ingest_docs_cli(args: LocalIngestDocsArgs) -> anyhow::Result< .await?; let merge_pipeline_handle = indexing_server_mailbox .ask_for_res(DetachMergePipeline { - pipeline_id: MergePipelineId::from(&pipeline_id), + pipeline_id: pipeline_id.merge_pipeline_id(), }) .await?; let indexing_pipeline_handle = indexing_server_mailbox @@ -618,7 +616,7 @@ pub async fn merge_cli(args: MergeArgs) -> anyhow::Result<()> { .await?; let pipeline_handle: ActorHandle = indexing_service_mailbox .ask_for_res(DetachMergePipeline { - pipeline_id: MergePipelineId::from(&pipeline_id), + pipeline_id: pipeline_id.merge_pipeline_id(), }) .await?; @@ -931,9 +929,8 @@ impl ThroughputCalculator { } async fn create_empty_cluster(config: &NodeConfig) -> anyhow::Result { - let node_id: NodeId = config.node_id.clone().into(); let self_node = ClusterMember { - node_id, + node_id: config.node_id.clone(), generation_id: quickwit_cluster::GenerationId::now(), is_ready: false, enabled_services: HashSet::new(), diff --git a/quickwit/quickwit-cli/tests/helpers.rs b/quickwit/quickwit-cli/tests/helpers.rs index d52e48034d..1ff7fb8234 100644 --- a/quickwit/quickwit-cli/tests/helpers.rs +++ b/quickwit/quickwit-cli/tests/helpers.rs @@ -33,6 +33,7 @@ use quickwit_common::uri::Uri; use quickwit_config::service::QuickwitService; use quickwit_metastore::{IndexMetadata, IndexMetadataResponseExt, MetastoreResolver}; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; +use quickwit_proto::types::IndexId; use quickwit_storage::{Storage, StorageResolver}; use reqwest::Url; use tempfile::{tempdir, TempDir}; @@ -120,7 +121,7 @@ pub struct TestEnv { pub cluster_endpoint: Url, pub index_config_uri: Uri, /// The index ID. - pub index_id: String, + pub index_id: IndexId, pub index_uri: Uri, pub rest_listen_port: u16, pub storage_resolver: StorageResolver, @@ -177,7 +178,7 @@ pub enum TestStorageType { /// Creates all necessary artifacts in a test environment. pub async fn create_test_env( - index_id: String, + index_id: IndexId, storage_type: TestStorageType, ) -> anyhow::Result { let temp_dir = tempdir()?; diff --git a/quickwit/quickwit-cluster/src/lib.rs b/quickwit/quickwit-cluster/src/lib.rs index 3cd08fdccc..7174bde9ab 100644 --- a/quickwit/quickwit-cluster/src/lib.rs +++ b/quickwit/quickwit-cluster/src/lib.rs @@ -39,7 +39,6 @@ use quickwit_common::metrics::IntCounter; use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; use quickwit_proto::indexing::CpuCapacity; -use quickwit_proto::types::NodeId; use time::OffsetDateTime; #[cfg(any(test, feature = "testsuite"))] @@ -129,7 +128,7 @@ pub async fn start_cluster_service(node_config: &NodeConfig) -> anyhow::Result for IndexConfig { #[derive(Clone, Debug, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct IndexConfigV0_8 { - pub index_id: String, + pub index_id: IndexId, #[schema(value_type = String)] #[serde(default)] pub index_uri: Option, diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index fd8d1a129e..bd58a139b6 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -27,6 +27,7 @@ use json_comments::StripComments; use once_cell::sync::Lazy; use quickwit_common::net::is_valid_hostname; use quickwit_common::uri::Uri; +use quickwit_proto::types::NodeIdRef; use regex::Regex; mod cluster_config; @@ -179,8 +180,8 @@ pub fn validate_index_id_pattern(pattern: &str, allow_negative: bool) -> anyhow: Ok(()) } -pub fn validate_node_id(node_id: &str) -> anyhow::Result<()> { - if !is_valid_hostname(node_id) { +pub fn validate_node_id(node_id: &NodeIdRef) -> anyhow::Result<()> { + if !is_valid_hostname(node_id.as_str()) { bail!( "node identifier `{node_id}` is invalid. node identifiers must be valid short \ hostnames (see RFC 1123)" diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index 841bbc3df6..aaf6bfaee9 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -32,6 +32,7 @@ use http::HeaderMap; use quickwit_common::net::HostAddr; use quickwit_common::uri::Uri; use quickwit_proto::indexing::CpuCapacity; +use quickwit_proto::types::NodeId; use serde::{Deserialize, Serialize}; use tracing::warn; @@ -395,7 +396,7 @@ impl Default for JaegerConfig { #[derive(Clone, Debug, Serialize)] pub struct NodeConfig { pub cluster_id: String, - pub node_id: String, + pub node_id: NodeId, pub enabled_services: HashSet, pub gossip_listen_addr: SocketAddr, pub grpc_listen_addr: SocketAddr, diff --git a/quickwit/quickwit-config/src/node_config/serialize.rs b/quickwit/quickwit-config/src/node_config/serialize.rs index ea3c629e8c..10d50159d9 100644 --- a/quickwit/quickwit-config/src/node_config/serialize.rs +++ b/quickwit/quickwit-config/src/node_config/serialize.rs @@ -27,6 +27,7 @@ use http::HeaderMap; use quickwit_common::net::{find_private_ip, get_short_hostname, Host}; use quickwit_common::new_coolid; use quickwit_common::uri::Uri; +use quickwit_proto::types::NodeId; use serde::{Deserialize, Serialize}; use tracing::{info, warn}; @@ -222,6 +223,8 @@ impl NodeConfigBuilder { mut self, env_vars: &HashMap, ) -> anyhow::Result { + let node_id = self.node_id.resolve(env_vars).map(NodeId::new)?; + let enabled_services = self .enabled_services .resolve(env_vars)? @@ -307,7 +310,7 @@ impl NodeConfigBuilder { let node_config = NodeConfig { cluster_id: self.cluster_id.resolve(env_vars)?, - node_id: self.node_id.resolve(env_vars)?, + node_id, enabled_services, gossip_listen_addr, grpc_listen_addr, @@ -414,6 +417,7 @@ impl RestConfigBuilder { pub fn node_config_for_test() -> NodeConfig { use quickwit_common::net::find_available_tcp_port; + let node_id = NodeId::new(default_node_id().unwrap()); let enabled_services = QuickwitService::supported_services(); let listen_address = Host::default(); let rest_listen_port = find_available_tcp_port().expect("OS should find an available port"); @@ -445,7 +449,7 @@ pub fn node_config_for_test() -> NodeConfig { }; NodeConfig { cluster_id: default_cluster_id().unwrap(), - node_id: default_node_id().unwrap(), + node_id, enabled_services, gossip_advertise_addr: gossip_listen_addr, grpc_advertise_addr: grpc_listen_addr, diff --git a/quickwit/quickwit-config/src/source_config/mod.rs b/quickwit/quickwit-config/src/source_config/mod.rs index 320f06317a..d1235df8a3 100644 --- a/quickwit/quickwit-config/src/source_config/mod.rs +++ b/quickwit/quickwit-config/src/source_config/mod.rs @@ -27,6 +27,7 @@ use bytes::Bytes; use quickwit_common::is_false; use quickwit_common::uri::Uri; use quickwit_proto::metastore::SourceType; +use quickwit_proto::types::SourceId; use serde::de::Error; use serde::{Deserialize, Deserializer, Serialize}; use serde_json::Value as JsonValue; @@ -53,7 +54,7 @@ pub const RESERVED_SOURCE_IDS: &[&str] = #[serde(into = "VersionedSourceConfig")] #[serde(try_from = "VersionedSourceConfig")] pub struct SourceConfig { - pub source_id: String, + pub source_id: SourceId, /// Number of indexing pipelines to run on a cluster for the source. pub num_pipelines: NonZeroUsize, diff --git a/quickwit/quickwit-config/src/source_config/serialize.rs b/quickwit/quickwit-config/src/source_config/serialize.rs index ae2c00bb41..7e6c834800 100644 --- a/quickwit/quickwit-config/src/source_config/serialize.rs +++ b/quickwit/quickwit-config/src/source_config/serialize.rs @@ -20,6 +20,7 @@ use std::num::NonZeroUsize; use anyhow::bail; +use quickwit_proto::types::SourceId; use serde::{Deserialize, Serialize}; use super::{TransformConfig, RESERVED_SOURCE_IDS}; @@ -74,7 +75,7 @@ impl SourceConfigForSerialization { /// TODO refactor #1065 fn validate_and_build(self) -> anyhow::Result { if !RESERVED_SOURCE_IDS.contains(&self.source_id.as_str()) { - validate_identifier("Source ID", &self.source_id)?; + validate_identifier("source", &self.source_id)?; } let num_pipelines = NonZeroUsize::new(self.num_pipelines) .ok_or_else(|| anyhow::anyhow!("`desired_num_pipelines` must be strictly positive"))?; @@ -174,7 +175,7 @@ fn default_source_enabled() -> bool { #[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct SourceConfigV0_7 { - pub source_id: String, + pub source_id: SourceId, #[serde( default = "default_max_num_pipelines_per_indexer", @@ -203,7 +204,7 @@ pub struct SourceConfigV0_7 { #[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct SourceConfigV0_8 { - pub source_id: String, + pub source_id: SourceId, #[serde(default = "default_num_pipelines")] pub num_pipelines: usize, diff --git a/quickwit/quickwit-indexing/failpoints/mod.rs b/quickwit/quickwit-indexing/failpoints/mod.rs index 8ea6b81b2c..0e468bc20e 100644 --- a/quickwit/quickwit-indexing/failpoints/mod.rs +++ b/quickwit/quickwit-indexing/failpoints/mod.rs @@ -53,9 +53,9 @@ use quickwit_metastore::{ ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitMetadata, SplitState, }; -use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::{ListSplitsRequest, MetastoreService}; -use quickwit_proto::types::{IndexUid, PipelineUid}; +use quickwit_proto::types::{IndexUid, NodeId}; use serde_json::Value as JsonValue; use tantivy::Directory; @@ -298,11 +298,10 @@ async fn test_merge_executor_controlled_directory_kill_switch() -> anyhow::Resul downloaded_splits_directory, tantivy_dirs, }; - let pipeline_id = IndexingPipelineId { + let pipeline_id = MergePipelineId { + node_id: NodeId::from("test-node"), index_uid: IndexUid::new_with_random_ulid(index_id), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), }; let universe = test_index_builder.universe(); diff --git a/quickwit/quickwit-indexing/src/actors/doc_processor.rs b/quickwit/quickwit-indexing/src/actors/doc_processor.rs index b3c7f84ee2..26b11155d8 100644 --- a/quickwit/quickwit-indexing/src/actors/doc_processor.rs +++ b/quickwit/quickwit-indexing/src/actors/doc_processor.rs @@ -33,6 +33,7 @@ use quickwit_opentelemetry::otlp::{ parse_otlp_logs_json, parse_otlp_logs_protobuf, parse_otlp_spans_json, parse_otlp_spans_protobuf, JsonLogIterator, JsonSpanIterator, OtlpLogsError, OtlpTracesError, }; +use quickwit_proto::types::{IndexId, SourceId}; use serde::Serialize; use serde_json::Value as JsonValue; use tantivy::schema::{Field, Value}; @@ -271,8 +272,8 @@ impl From> for JsonDocIterator { #[derive(Debug, Serialize)] pub struct DocProcessorCounters { - index_id: String, - source_id: String, + index_id: IndexId, + source_id: SourceId, /// Overall number of documents received, partitioned /// into 4 categories: /// - number of docs that could not be parsed. @@ -292,7 +293,7 @@ pub struct DocProcessorCounters { } impl DocProcessorCounters { - pub fn new(index_id: String, source_id: String) -> Self { + pub fn new(index_id: IndexId, source_id: SourceId) -> Self { Self { index_id, source_id, @@ -382,8 +383,8 @@ pub struct DocProcessor { impl DocProcessor { pub fn try_new( - index_id: String, - source_id: String, + index_id: IndexId, + source_id: SourceId, doc_mapper: Arc, indexer_mailbox: Mailbox, transform_config_opt: Option, diff --git a/quickwit/quickwit-indexing/src/actors/indexer.rs b/quickwit/quickwit-indexing/src/actors/indexer.rs index 93b8bb4706..22de7b3029 100644 --- a/quickwit/quickwit-indexing/src/actors/indexer.rs +++ b/quickwit/quickwit-indexing/src/actors/indexer.rs @@ -706,7 +706,7 @@ mod tests { use quickwit_proto::metastore::{ EmptyResponse, LastDeleteOpstampResponse, MockMetastoreService, }; - use quickwit_proto::types::{IndexUid, PipelineUid}; + use quickwit_proto::types::{IndexUid, NodeId, PipelineUid}; use tantivy::{doc, DateTime}; use super::*; @@ -747,7 +747,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper = Arc::new(default_doc_mapper_for_test()); @@ -885,7 +885,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: index_uid.clone(), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper = Arc::new(default_doc_mapper_for_test()); @@ -962,7 +962,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper = Arc::new(default_doc_mapper_for_test()); @@ -1046,7 +1046,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper = Arc::new(default_doc_mapper_for_test()); @@ -1134,7 +1134,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper = Arc::new(default_doc_mapper_for_test()); @@ -1214,7 +1214,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper: Arc = Arc::new( @@ -1315,7 +1315,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper: Arc = @@ -1386,7 +1386,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper: Arc = @@ -1458,7 +1458,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper: Arc = @@ -1523,7 +1523,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper: Arc = @@ -1584,7 +1584,7 @@ mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let doc_mapper = Arc::new(default_doc_mapper_for_test()); diff --git a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs index 8e4a98f892..f38ccaa6b6 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_pipeline.rs @@ -605,7 +605,7 @@ mod tests { EmptyResponse, IndexMetadataResponse, LastDeleteOpstampResponse, MetastoreError, MockMetastoreService, }; - use quickwit_proto::types::{IndexUid, PipelineUid}; + use quickwit_proto::types::{IndexUid, NodeId, PipelineUid}; use quickwit_storage::RamStorage; use super::{IndexingPipeline, *}; @@ -626,7 +626,7 @@ mod tests { mut num_fails: usize, test_file: &str, ) -> anyhow::Result<()> { - let node_id = "test-node".to_string(); + let node_id = NodeId::from("test-node"); let index_uid = IndexUid::for_test("test-index", 2); let pipeline_id = IndexingPipelineId { node_id, @@ -745,7 +745,7 @@ mod tests { } async fn indexing_pipeline_simple(test_file: &str) -> anyhow::Result<()> { - let node_id = "test-node".to_string(); + let node_id = NodeId::from("test-node"); let index_uid: IndexUid = IndexUid::for_test("test-index", 1); let pipeline_id = IndexingPipelineId { node_id, @@ -850,7 +850,7 @@ mod tests { #[tokio::test] async fn test_merge_pipeline_does_not_stop_on_indexing_pipeline_failure() { - let node_id = "test-node".to_string(); + let node_id = NodeId::from("test-node"); let pipeline_id = IndexingPipelineId { node_id, index_uid: IndexUid::new_with_random_ulid("test-index"), @@ -891,7 +891,7 @@ mod tests { let storage = Arc::new(RamStorage::default()); let split_store = IndexingSplitStore::create_without_local_store_for_test(storage.clone()); let merge_pipeline_params = MergePipelineParams { - pipeline_id: pipeline_id.clone(), + pipeline_id: pipeline_id.merge_pipeline_id(), doc_mapper: doc_mapper.clone(), indexing_directory: TempDirectory::for_test(), metastore: metastore.clone(), @@ -902,7 +902,7 @@ mod tests { merge_scheduler_service: universe.get_or_spawn_one(), event_broker: Default::default(), }; - let merge_pipeline = MergePipeline::new(merge_pipeline_params, universe.spawn_ctx()); + let merge_pipeline = MergePipeline::new(merge_pipeline_params, None, universe.spawn_ctx()); let merge_planner_mailbox = merge_pipeline.merge_planner_mailbox().clone(); let (_merge_pipeline_mailbox, merge_pipeline_handler) = universe.spawn_builder().spawn(merge_pipeline); @@ -952,7 +952,7 @@ mod tests { } async fn indexing_pipeline_all_failures_handling(test_file: &str) -> anyhow::Result<()> { - let node_id = "test-node".to_string(); + let node_id = NodeId::from("test-node"); let index_uid: IndexUid = IndexUid::for_test("test-index", 2); let pipeline_id = IndexingPipelineId { node_id, diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 1a988f5661..321b0be877 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -18,12 +18,13 @@ // along with this program. If not, see . use std::collections::{HashMap, HashSet}; -use std::fmt::{Debug, Display, Formatter}; +use std::fmt::{Debug, Formatter}; use std::path::PathBuf; use std::sync::Arc; use anyhow::Context; use async_trait::async_trait; +use futures::TryStreamExt; use itertools::Itertools; use quickwit_actors::{ Actor, ActorContext, ActorExitStatus, ActorHandle, ActorState, Handler, Healthz, Mailbox, @@ -43,19 +44,22 @@ use quickwit_ingest::{ }; use quickwit_metastore::{ IndexMetadata, IndexMetadataResponseExt, IndexesMetadataResponseExt, - ListIndexesMetadataResponseExt, + ListIndexesMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, + SplitMetadata, SplitState, }; use quickwit_proto::indexing::{ ApplyIndexingPlanRequest, ApplyIndexingPlanResponse, IndexingError, IndexingPipelineId, - IndexingTask, PipelineMetrics, + IndexingTask, MergePipelineId, PipelineMetrics, }; use quickwit_proto::metastore::{ IndexMetadataRequest, IndexMetadataSubrequest, IndexesMetadataRequest, - ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, + ListIndexesMetadataRequest, ListSplitsRequest, MetastoreResult, MetastoreService, + MetastoreServiceClient, }; -use quickwit_proto::types::{IndexId, IndexUid, PipelineUid, ShardId}; +use quickwit_proto::types::{IndexId, IndexUid, NodeId, PipelineUid, ShardId}; use quickwit_storage::StorageResolver; use serde::{Deserialize, Serialize}; +use time::OffsetDateTime; use tokio::sync::Semaphore; use tracing::{debug, error, info, warn}; @@ -79,27 +83,6 @@ pub struct IndexingServiceCounters { pub num_delete_queue_failures: usize, } -#[derive(Debug, Clone, Hash, Eq, PartialEq)] -pub struct MergePipelineId { - index_uid: IndexUid, - source_id: String, -} - -impl Display for MergePipelineId { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - write!(f, "merge:{}:{}", self.index_uid, self.source_id) - } -} - -impl From<&IndexingPipelineId> for MergePipelineId { - fn from(pipeline_id: &IndexingPipelineId) -> Self { - MergePipelineId { - index_uid: pipeline_id.index_uid.clone(), - source_id: pipeline_id.source_id.clone(), - } - } -} - struct MergePipelineHandle { mailbox: Mailbox, handle: ActorHandle, @@ -118,7 +101,7 @@ struct PipelineHandle { /// with the target situation, and spawning/shutting down the indexing pipelines that /// are respectively missing or extranumerous. pub struct IndexingService { - node_id: String, + node_id: NodeId, indexing_root_directory: PathBuf, queue_dir_path: PathBuf, cluster: Cluster, @@ -151,7 +134,7 @@ impl Debug for IndexingService { impl IndexingService { #[allow(clippy::too_many_arguments)] pub async fn new( - node_id: String, + node_id: NodeId, data_dir_path: PathBuf, indexer_config: IndexerConfig, num_blocking_threads: usize, @@ -262,7 +245,7 @@ impl IndexingService { pipeline_uid, }; let index_config = index_metadata.into_index_config(); - self.spawn_pipeline_inner(ctx, pipeline_id.clone(), index_config, source_config) + self.spawn_pipeline_inner(ctx, pipeline_id.clone(), index_config, source_config, None) .await?; Ok(pipeline_id) } @@ -270,22 +253,23 @@ impl IndexingService { async fn spawn_pipeline_inner( &mut self, ctx: &ActorContext, - pipeline_id: IndexingPipelineId, + indexing_pipeline_id: IndexingPipelineId, index_config: IndexConfig, source_config: SourceConfig, + immature_splits_opt: Option>, ) -> Result<(), IndexingError> { if self .indexing_pipelines - .contains_key(&pipeline_id.pipeline_uid) + .contains_key(&indexing_pipeline_id.pipeline_uid) { - let message = format!("pipeline `{pipeline_id}` already exists"); + let message = format!("pipeline `{indexing_pipeline_id}` already exists"); return Err(IndexingError::Internal(message)); } - let pipeline_uid_str = pipeline_id.pipeline_uid.to_string(); + let pipeline_uid_str = indexing_pipeline_id.pipeline_uid.to_string(); let indexing_directory = temp_dir::Builder::default() - .join(&pipeline_id.index_uid.index_id) - .join(&pipeline_id.index_uid.incarnation_id.to_string()) - .join(&pipeline_id.source_id) + .join(&indexing_pipeline_id.index_uid.index_id) + .join(&indexing_pipeline_id.index_uid.incarnation_id.to_string()) + .join(&indexing_pipeline_id.source_id) .join(&pipeline_uid_str) .tempdir_in(&self.indexing_root_directory) .map_err(|error| { @@ -307,8 +291,9 @@ impl IndexingService { let doc_mapper = build_doc_mapper(&index_config.doc_mapping, &index_config.search_settings) .map_err(|error| IndexingError::Internal(error.to_string()))?; + let merge_pipeline_id = indexing_pipeline_id.merge_pipeline_id(); let merge_pipeline_params = MergePipelineParams { - pipeline_id: pipeline_id.clone(), + pipeline_id: merge_pipeline_id.clone(), doc_mapper: doc_mapper.clone(), indexing_directory: indexing_directory.clone(), metastore: self.metastore.clone(), @@ -319,11 +304,8 @@ impl IndexingService { max_concurrent_split_uploads: self.max_concurrent_split_uploads, event_broker: self.event_broker.clone(), }; - - let merge_planner_mailbox = self - .get_or_create_merge_pipeline(merge_pipeline_params, ctx) - .await?; - + let merge_planner_mailbox = + self.get_or_create_merge_pipeline(merge_pipeline_params, immature_splits_opt, ctx)?; // The concurrent uploads budget is split in 2: 1/2 for the indexing pipeline, 1/2 for the // merge pipeline. let max_concurrent_split_uploads_index = (self.max_concurrent_split_uploads / 2).max(1); @@ -331,7 +313,7 @@ impl IndexingService { (self.max_concurrent_split_uploads - max_concurrent_split_uploads_index).max(1); let pipeline_params = IndexingPipelineParams { - pipeline_id: pipeline_id.clone(), + pipeline_id: indexing_pipeline_id.clone(), metastore: self.metastore.clone(), storage, // Indexing-related parameters @@ -359,10 +341,10 @@ impl IndexingService { let pipeline_handle = PipelineHandle { mailbox: pipeline_mailbox, handle: pipeline_handle, - indexing_pipeline_id: pipeline_id.clone(), + indexing_pipeline_id: indexing_pipeline_id.clone(), }; self.indexing_pipelines - .insert(pipeline_id.pipeline_uid, pipeline_handle); + .insert(indexing_pipeline_id.pipeline_uid, pipeline_handle); self.counters.num_running_pipelines += 1; Ok(()) } @@ -410,6 +392,67 @@ impl IndexingService { Ok(indexes_metadata) } + /// Fetches the immature splits candidates for merge for all the indexing pipelines for which a + /// merge pipeline is not running. + async fn fetch_immature_splits( + &mut self, + indexing_pipeline_ids: &[IndexingPipelineId], + ctx: &ActorContext, + ) -> MetastoreResult>> { + let mut index_uids = Vec::new(); + + for indexing_pipeline_id in indexing_pipeline_ids { + let merge_pipeline_id = indexing_pipeline_id.merge_pipeline_id(); + + if !self.merge_pipeline_handles.contains_key(&merge_pipeline_id) { + index_uids.push(merge_pipeline_id.index_uid); + } + } + if index_uids.is_empty() { + return Ok(Default::default()); + } + index_uids.sort_unstable(); + index_uids.dedup(); + + let list_splits_query = ListSplitsQuery::try_from_index_uids(index_uids) + .expect("`index_uids` should not be empty") + .with_node_id(self.node_id.clone()) + .with_split_state(SplitState::Published) + .retain_immature(OffsetDateTime::now_utc()); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(&list_splits_query)?; + + let mut immature_splits_stream = ctx + .protect_future(self.metastore.list_splits(list_splits_request)) + .await?; + + let mut per_merge_pipeline_immature_splits: HashMap> = + indexing_pipeline_ids + .iter() + .map(|indexing_pipeline_id| (indexing_pipeline_id.merge_pipeline_id(), Vec::new())) + .collect(); + + let mut num_immature_splits = 0usize; + + while let Some(list_splits_response) = immature_splits_stream.try_next().await? { + for split_metadata in list_splits_response.deserialize_splits_metadata()? { + num_immature_splits += 1; + + let merge_pipeline_id = MergePipelineId { + node_id: self.node_id.clone(), + index_uid: split_metadata.index_uid.clone(), + source_id: split_metadata.source_id.clone(), + }; + per_merge_pipeline_immature_splits + .entry(merge_pipeline_id) + .or_default() + .push(split_metadata); + } + } + info!("fetched {num_immature_splits} splits candidates for merge"); + Ok(per_merge_pipeline_immature_splits) + } + async fn handle_supervise(&mut self) -> Result<(), ActorExitStatus> { self.indexing_pipelines .retain(|pipeline_uid, pipeline_handle| { @@ -440,7 +483,7 @@ impl IndexingService { let merge_pipelines_to_retain: HashSet = self .indexing_pipelines .values() - .map(|pipeline_handle| MergePipelineId::from(&pipeline_handle.indexing_pipeline_id)) + .map(|pipeline_handle| pipeline_handle.indexing_pipeline_id.merge_pipeline_id()) .collect(); let merge_pipelines_to_shutdown: Vec = self @@ -490,18 +533,21 @@ impl IndexingService { Ok(()) } - async fn get_or_create_merge_pipeline( + fn get_or_create_merge_pipeline( &mut self, merge_pipeline_params: MergePipelineParams, + immature_splits_opt: Option>, ctx: &ActorContext, ) -> Result, IndexingError> { - let merge_pipeline_id = MergePipelineId::from(&merge_pipeline_params.pipeline_id); - if let Some(merge_pipeline_mailbox_handle) = - self.merge_pipeline_handles.get(&merge_pipeline_id) + if let Some(merge_pipeline_mailbox_handle) = self + .merge_pipeline_handles + .get(&merge_pipeline_params.pipeline_id) { return Ok(merge_pipeline_mailbox_handle.mailbox.clone()); } - let merge_pipeline = MergePipeline::new(merge_pipeline_params, ctx.spawn_ctx()); + let merge_pipeline_id = merge_pipeline_params.pipeline_id.clone(); + let merge_pipeline = + MergePipeline::new(merge_pipeline_params, immature_splits_opt, ctx.spawn_ctx()); let merge_planner_mailbox = merge_pipeline.merge_planner_mailbox().clone(); let (_pipeline_mailbox, pipeline_handle) = ctx.spawn_actor().spawn(merge_pipeline); let merge_pipeline_mailbox_handle = MergePipelineHandle { @@ -619,6 +665,9 @@ impl IndexingService { .map(|index_metadata| (index_metadata.index_uid.clone(), index_metadata)) .collect(); + let mut per_merge_pipeline_immature_splits: HashMap> = + self.fetch_immature_splits(pipelines_to_spawn, ctx).await?; + let mut spawn_pipeline_failures: Vec = Vec::new(); for pipeline_to_spawn in pipelines_to_spawn { @@ -628,12 +677,17 @@ impl IndexingService { if let Some(source_config) = index_metadata.sources.get(&pipeline_to_spawn.source_id) { + let merge_pipeline_id = pipeline_to_spawn.merge_pipeline_id(); + let immature_splits_opt = + per_merge_pipeline_immature_splits.remove(&merge_pipeline_id); + if let Err(error) = self .spawn_pipeline_inner( ctx, pipeline_to_spawn.clone(), index_metadata.index_config.clone(), source_config.clone(), + immature_splits_opt, ) .await { @@ -739,6 +793,9 @@ impl IndexingService { .collect(); debug!(queues=?queues, "list ingest API queues"); + if queues.is_empty() { + return Ok(()); + } let indexes_metadata = self .metastore .list_indexes_metadata(ListIndexesMetadataRequest::all()) @@ -905,6 +962,7 @@ impl Handler for IndexingService { } } +#[derive(Debug)] struct IndexingPipelineDiff { pipelines_to_shutdown: Vec, pipelines_to_spawn: Vec, @@ -927,12 +985,13 @@ mod tests { use quickwit_ingest::{init_ingest_api, CreateQueueIfNotExistsRequest}; use quickwit_metastore::{ metastore_for_test, AddSourceRequestExt, CreateIndexRequestExt, - ListIndexesMetadataResponseExt, + ListIndexesMetadataResponseExt, Split, }; use quickwit_proto::indexing::IndexingTask; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, IndexMetadataResponse, - ListIndexesMetadataResponse, MockMetastoreService, + IndexesMetadataResponse, ListIndexesMetadataResponse, ListSplitsResponse, + MockMetastoreService, }; use super::*; @@ -953,7 +1012,7 @@ mod tests { .unwrap(); let merge_scheduler_mailbox: Mailbox = universe.get_or_spawn_one(); let indexing_server = IndexingService::new( - "test-node".to_string(), + NodeId::from("test-node"), data_dir_path.to_path_buf(), indexer_config, num_blocking_threads, @@ -1062,7 +1121,7 @@ mod tests { pipeline_handle.kill().await; let _merge_pipeline = indexing_service .ask_for_res(DetachMergePipeline { - pipeline_id: MergePipelineId::from(&pipeline_id), + pipeline_id: pipeline_id.merge_pipeline_id(), }) .await .unwrap(); @@ -1406,7 +1465,7 @@ mod tests { .unwrap(); let merge_scheduler_service = universe.get_or_spawn_one(); let indexing_server = IndexingService::new( - "test-node".to_string(), + NodeId::from("test-node"), data_dir_path, indexer_config, num_blocking_threads, @@ -1608,7 +1667,7 @@ mod tests { let storage_resolver = StorageResolver::unconfigured(); let merge_scheduler_service: Mailbox = universe.get_or_spawn_one(); let mut indexing_server = IndexingService::new( - "test-ingest-api-gc-node".to_string(), + NodeId::from("test-ingest-api-gc-node"), data_dir_path, indexer_config, num_blocking_threads, @@ -1638,4 +1697,125 @@ mod tests { universe.assert_quit().await; } + + #[tokio::test] + async fn test_indexing_service_apply_indexing_plan_batches_metastore_calls() { + let temp_dir = tempfile::tempdir().unwrap(); + let universe = Universe::new(); + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_index_metadata() + .withf(|request| request.index_id.as_ref().unwrap() == "test-index-0") + .return_once(|_request| { + let index_metadata_0 = + IndexMetadata::for_test("test-index-0", "ram:///indexes/test-index-0"); + let response = + IndexMetadataResponse::try_from_index_metadata(&index_metadata_0).unwrap(); + Ok(response) + }); + mock_metastore + .expect_indexes_metadata() + .withf(|request| { + let index_uids: Vec<&IndexUid> = request + .subrequests + .iter() + .flat_map(|subrequest| &subrequest.index_uid) + .sorted() + .collect(); + + index_uids == [&("test-index-1", 0), &("test-index-2", 0)] + }) + .return_once(|_request| { + let source_config = SourceConfig::for_test("test-source", SourceParams::void()); + + let mut index_metadata_1 = + IndexMetadata::for_test("test-index-1", "ram:///indexes/test-index-1"); + index_metadata_1.add_source(source_config.clone()).unwrap(); + + let mut index_metadata_2 = + IndexMetadata::for_test("test-index-2", "ram:///indexes/test-index-2"); + index_metadata_2.add_source(source_config).unwrap(); + + let indexes_metadata = vec![index_metadata_1, index_metadata_2]; + let failures = Vec::new(); + let response = IndexesMetadataResponse::for_test(indexes_metadata, failures); + Ok(response) + }); + mock_metastore + .expect_list_splits() + .withf(|request| { + let list_splits_query = request.deserialize_list_splits_query().unwrap(); + list_splits_query.index_uids == [("test-index-0", 0)] + }) + .return_once(|_request| Ok(ServiceStream::empty())); + mock_metastore + .expect_list_splits() + .withf(|request| { + let list_splits_query = request.deserialize_list_splits_query().unwrap(); + list_splits_query.index_uids == [("test-index-1", 0), ("test-index-2", 0)] + }) + .return_once(|_request| { + let splits = vec![Split { + split_metadata: SplitMetadata::for_test("test-split".to_string()), + split_state: SplitState::Published, + update_timestamp: 0, + publish_timestamp: Some(0), + }]; + let list_splits_response = ListSplitsResponse::try_from_splits(splits).unwrap(); + let response = ServiceStream::from(vec![Ok(list_splits_response)]); + Ok(response) + }); + + let transport = ChannelTransport::default(); + let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) + .await + .unwrap(); + let (indexing_service, _indexing_service_handle) = spawn_indexing_service_for_test( + temp_dir.path(), + &universe, + MetastoreServiceClient::from_mock(mock_metastore), + cluster, + ) + .await; + + let source_config = SourceConfig::for_test("test-source", SourceParams::void()); + + indexing_service + .ask_for_res(SpawnPipeline { + index_id: "test-index-0".to_string(), + source_config, + pipeline_uid: PipelineUid::for_test(0), + }) + .await + .unwrap(); + + indexing_service + .ask_for_res(ApplyIndexingPlanRequest { + indexing_tasks: vec![ + IndexingTask { + index_uid: Some(IndexUid::for_test("test-index-0", 0)), + source_id: "test-source".to_string(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(0)), + }, + IndexingTask { + index_uid: Some(IndexUid::for_test("test-index-1", 0)), + source_id: "test-source".to_string(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(1)), + }, + IndexingTask { + index_uid: Some(IndexUid::for_test("test-index-2", 0)), + source_id: "test-source".to_string(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(2)), + }, + ], + }) + .await + .unwrap(); + + universe.assert_quit().await; + } } diff --git a/quickwit/quickwit-indexing/src/actors/merge_executor.rs b/quickwit/quickwit-indexing/src/actors/merge_executor.rs index 19b8208be3..64b59cb2aa 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_executor.rs @@ -34,12 +34,12 @@ use quickwit_common::temp_dir::TempDirectory; use quickwit_directories::UnionDirectory; use quickwit_doc_mapper::DocMapper; use quickwit_metastore::SplitMetadata; -use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::{ DeleteTask, ListDeleteTasksRequest, MarkSplitsForDeletionRequest, MetastoreService, MetastoreServiceClient, }; -use quickwit_proto::types::PipelineUid; +use quickwit_proto::types::{NodeId, SplitId}; use quickwit_query::get_quickwit_fastfield_normalizer_manager; use quickwit_query::query_ast::QueryAst; use tantivy::directory::{Advice, DirectoryClone, MmapDirectory, RamDirectory}; @@ -56,7 +56,7 @@ use crate::models::{IndexedSplit, IndexedSplitBatch, MergeScratch, PublishLock, #[derive(Clone)] pub struct MergeExecutor { - pipeline_id: IndexingPipelineId, + pipeline_id: MergePipelineId, metastore: MetastoreServiceClient, doc_mapper: Arc, io_controls: IoControls, @@ -233,15 +233,15 @@ pub fn combine_partition_ids(splits: &[SplitMetadata]) -> u64 { } pub fn merge_split_attrs( - merge_split_id: String, - pipeline_id: &IndexingPipelineId, + pipeline_id: MergePipelineId, + merge_split_id: SplitId, splits: &[SplitMetadata], ) -> SplitAttrs { let partition_id = combine_partition_ids_aux(splits.iter().map(|split| split.partition_id)); let time_range: Option> = merge_time_range(splits); let uncompressed_docs_size_in_bytes = sum_doc_sizes_in_bytes(splits); let num_docs = sum_num_docs(splits); - let replaced_split_ids: Vec = splits + let replaced_split_ids: Vec = splits .iter() .map(|split| split.split_id().to_string()) .collect(); @@ -251,9 +251,11 @@ pub fn merge_split_attrs( .min() .unwrap_or(0); SplitAttrs { + node_id: pipeline_id.node_id.clone(), + index_uid: pipeline_id.index_uid.clone(), + source_id: pipeline_id.source_id.clone(), split_id: merge_split_id, partition_id, - pipeline_id: pipeline_id.clone(), replaced_split_ids, time_range, num_docs, @@ -273,7 +275,7 @@ fn max_merge_ops(splits: &[SplitMetadata]) -> usize { impl MergeExecutor { pub fn new( - pipeline_id: IndexingPipelineId, + pipeline_id: MergePipelineId, metastore: MetastoreServiceClient, doc_mapper: Arc, io_controls: IoControls, @@ -290,7 +292,7 @@ impl MergeExecutor { async fn process_merge( &mut self, - merge_split_id: String, + merge_split_id: SplitId, splits: Vec, tantivy_dirs: Vec>, merge_scratch_directory: TempDirectory, @@ -322,7 +324,7 @@ impl MergeExecutor { )?; ctx.record_progress(); - let split_attrs = merge_split_attrs(merge_split_id, &self.pipeline_id, &splits); + let split_attrs = merge_split_attrs(self.pipeline_id.clone(), merge_split_id, &splits); Ok(IndexedSplit { split_attrs, index: merged_index, @@ -333,7 +335,7 @@ impl MergeExecutor { async fn process_delete_and_merge( &mut self, - merge_split_id: String, + merge_split_id: SplitId, split: SplitMetadata, tantivy_dirs: Vec>, merge_scratch_directory: TempDirectory, @@ -429,18 +431,13 @@ impl MergeExecutor { } else { None }; - - let index_pipeline_id = IndexingPipelineId { - index_uid: split.index_uid, - node_id: split.node_id.clone(), - pipeline_uid: PipelineUid::new(), - source_id: split.source_id.clone(), - }; let indexed_split = IndexedSplit { split_attrs: SplitAttrs { + node_id: NodeId::new(split.node_id), + index_uid: split.index_uid, + source_id: split.source_id, split_id: merge_split_id, partition_id: split.partition_id, - pipeline_id: index_pipeline_id, replaced_split_ids: vec![split.split_id.clone()], time_range, num_docs, @@ -588,13 +585,6 @@ mod tests { "#; let test_sandbox = TestSandbox::create("test-index", doc_mapping_yaml, "", &["body"]).await?; - let index_uid = test_sandbox.index_uid(); - let pipeline_id = IndexingPipelineId { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::for_test(0u128), - }; for split_id in 0..4 { let single_doc = std::iter::once( serde_json::json!({"body ": format!("split{split_id}"), "ts": 1631072713u64 + split_id }), @@ -602,7 +592,8 @@ mod tests { test_sandbox.add_documents(single_doc).await?; } let mut metastore = test_sandbox.metastore(); - let list_splits_request = ListSplitsRequest::try_from_index_uid(index_uid).unwrap(); + let index_uid = test_sandbox.index_uid(); + let list_splits_request = ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap(); let split_metas: Vec = metastore .list_splits(list_splits_request) .await @@ -632,11 +623,16 @@ mod tests { merge_scratch_directory, downloaded_splits_directory, }; + let pipeline_id = MergePipelineId { + node_id: test_sandbox.node_id(), + index_uid, + source_id: test_sandbox.source_id(), + }; let (merge_packager_mailbox, merge_packager_inbox) = test_sandbox.universe().create_test_mailbox(); let merge_executor = MergeExecutor::new( pipeline_id, - metastore, + test_sandbox.metastore(), test_sandbox.doc_mapper(), IoControls::default(), merge_packager_mailbox, @@ -700,7 +696,6 @@ mod tests { result_docs: Vec, ) -> anyhow::Result<()> { quickwit_common::setup_logging_for_tests(); - let universe = Universe::with_accelerated_time(); let doc_mapping_yaml = r#" field_mappings: - name: body @@ -713,15 +708,9 @@ mod tests { timestamp_field: ts "#; let test_sandbox = TestSandbox::create(index_id, doc_mapping_yaml, "", &["body"]).await?; - let index_uid = test_sandbox.index_uid(); - let pipeline_id = IndexingPipelineId { - index_uid: index_uid.clone(), - node_id: "unknown".to_string(), - pipeline_uid: PipelineUid::for_test(0u128), - source_id: "unknown".to_string(), - }; test_sandbox.add_documents(docs).await?; let mut metastore = test_sandbox.metastore(); + let index_uid = test_sandbox.index_uid(); metastore .create_delete_task(DeleteQuery { index_uid: Some(index_uid.clone()), @@ -778,6 +767,12 @@ mod tests { merge_scratch_directory, downloaded_splits_directory, }; + let pipeline_id = MergePipelineId { + node_id: test_sandbox.node_id(), + index_uid: test_sandbox.index_uid(), + source_id: test_sandbox.source_id(), + }; + let universe = Universe::with_accelerated_time(); let (merge_packager_mailbox, merge_packager_inbox) = universe.create_test_mailbox(); let delete_task_executor = MergeExecutor::new( pipeline_id, @@ -838,17 +833,17 @@ mod tests { } else { assert!(packager_msgs.is_empty()); let mut metastore = test_sandbox.metastore(); - assert!(metastore + let index_uid = test_sandbox.index_uid(); + let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await .unwrap() .collect_splits() .await - .unwrap() - .into_iter() - .all( - |split| split.split_state == quickwit_metastore::SplitState::MarkedForDeletion - )); + .unwrap(); + assert!(splits.iter().all( + |split| split.split_state == quickwit_metastore::SplitState::MarkedForDeletion + )); } test_sandbox.assert_quit().await; universe.assert_quit().await; diff --git a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs index 3f29c5d6c6..acccf69fc2 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_pipeline.rs @@ -31,11 +31,12 @@ use quickwit_common::temp_dir::TempDirectory; use quickwit_common::KillSwitch; use quickwit_doc_mapper::DocMapper; use quickwit_metastore::{ - ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitState, + ListSplitsQuery, ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, SplitMetadata, + SplitState, }; -use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::{ - ListSplitsRequest, MetastoreError, MetastoreService, MetastoreServiceClient, + ListSplitsRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, }; use time::OffsetDateTime; use tokio::sync::Semaphore; @@ -96,6 +97,8 @@ pub struct MergePipeline { statistics: MergeStatistics, handles_opt: Option, kill_switch: KillSwitch, + /// Immature splits passed to the merge planner the first time the pipeline is spawned. + initial_immature_splits_opt: Option>, } #[async_trait] @@ -120,7 +123,11 @@ impl Actor for MergePipeline { impl MergePipeline { // TODO improve API. Maybe it could take a spawnbuilder as argument, hence removing the need // for a public create_mailbox / MessageCount. - pub fn new(params: MergePipelineParams, spawn_ctx: &SpawnContext) -> Self { + pub fn new( + params: MergePipelineParams, + initial_immature_splits_opt: Option>, + spawn_ctx: &SpawnContext, + ) -> Self { let (merge_planner_mailbox, merge_planner_inbox) = spawn_ctx .create_mailbox::("MergePlanner", MergePlanner::queue_capacity()); Self { @@ -131,6 +138,7 @@ impl MergePipeline { statistics: MergeStatistics::default(), merge_planner_inbox, merge_planner_mailbox, + initial_immature_splits_opt, } } @@ -160,6 +168,7 @@ impl MergePipeline { let mut healthy_actors: Vec<&str> = Default::default(); let mut failure_or_unhealthy_actors: Vec<&str> = Default::default(); let mut success_actors: Vec<&str> = Default::default(); + for supervisable in self.supervisables() { match supervisable.check_health(check_for_progress) { Health::Healthy => { @@ -174,35 +183,37 @@ impl MergePipeline { } } } - if !failure_or_unhealthy_actors.is_empty() { error!( - pipeline_id=?self.params.pipeline_id, + index_uid=%self.params.pipeline_id.index_uid, + source_id=%self.params.pipeline_id.source_id, generation=self.generation(), healthy_actors=?healthy_actors, failed_or_unhealthy_actors=?failure_or_unhealthy_actors, success_actors=?success_actors, - "Merge pipeline failure." + "merge pipeline failed" ); return Health::FailureOrUnhealthy; } if healthy_actors.is_empty() { // All the actors finished successfully. info!( - pipeline_id=?self.params.pipeline_id, + index_uid=%self.params.pipeline_id.index_uid, + source_id=%self.params.pipeline_id.source_id, generation=self.generation(), - "Merge pipeline success." + "merge pipeline completed successfully" ); return Health::Success; } // No error at this point and there are still some actors running. debug!( - pipeline_id=?self.params.pipeline_id, + index_uid=%self.params.pipeline_id.index_uid, + source_id=%self.params.pipeline_id.source_id, generation=self.generation(), healthy_actors=?healthy_actors, failed_or_unhealthy_actors=?failure_or_unhealthy_actors, success_actors=?success_actors, - "Merge pipeline running." + "merge pipeline is running and healthy" ); Health::Healthy } @@ -212,7 +223,7 @@ impl MergePipeline { } // TODO: Should return an error saying whether we can retry or not. - #[instrument(name="spawn_merge_pipeline", level="info", skip_all, fields(index=self.params.pipeline_id.index_uid.index_id, gen=self.generation()))] + #[instrument(name="spawn_merge_pipeline", level="info", skip_all, fields(index_uid=%self.params.pipeline_id.index_uid, generation=self.generation()))] async fn spawn_pipeline(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { let _spawn_pipeline_permit = ctx .protect_future(SPAWN_PIPELINE_SEMAPHORE.acquire()) @@ -223,27 +234,13 @@ impl MergePipeline { self.kill_switch = ctx.kill_switch().child(); info!( - index_id=%self.params.pipeline_id.index_uid.index_id, + index_uid=%self.params.pipeline_id.index_uid, source_id=%self.params.pipeline_id.source_id, - pipeline_uid=%self.params.pipeline_id.pipeline_uid, root_dir=%self.params.indexing_directory.path().display(), merge_policy=?self.params.merge_policy, "spawning merge pipeline", ); - let query = ListSplitsQuery::for_index(self.params.pipeline_id.index_uid.clone()) - .with_split_state(SplitState::Published) - .retain_immature(OffsetDateTime::now_utc()); - let list_splits_request = ListSplitsRequest::try_from_list_splits_query(&query)?; - let published_splits_stream = ctx - .protect_future(self.params.metastore.list_splits(list_splits_request)) - .await?; - let published_splits_metadata = ctx - .protect_future(published_splits_stream.collect_splits_metadata()) - .await?; - info!( - num_splits = published_splits_metadata.len(), - "loaded list of published splits" - ); + let immature_splits = self.fetch_immature_splits(ctx).await?; // Merge publisher let merge_publisher = Publisher::new( @@ -329,8 +326,8 @@ impl MergePipeline { // Merge planner let merge_planner = MergePlanner::new( - self.params.pipeline_id.clone(), - published_splits_metadata, + &self.params.pipeline_id, + immature_splits, self.params.merge_policy.clone(), merge_split_downloader_mailbox, self.params.merge_scheduler_service.clone(), @@ -418,6 +415,36 @@ impl MergePipeline { } Ok(()) } + + async fn fetch_immature_splits( + &mut self, + ctx: &ActorContext, + ) -> MetastoreResult> { + if let Some(immature_splits) = self.initial_immature_splits_opt.take() { + return Ok(immature_splits); + } + let index_uid = self.params.pipeline_id.index_uid.clone(); + let node_id = self.params.pipeline_id.node_id.clone(); + let list_splits_query = ListSplitsQuery::for_index(index_uid) + .with_node_id(node_id) + .with_split_state(SplitState::Published) + .retain_immature(OffsetDateTime::now_utc()); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(&list_splits_query)?; + let immature_splits_stream = ctx + .protect_future(self.params.metastore.list_splits(list_splits_request)) + .await?; + let immature_splits = ctx + .protect_future(immature_splits_stream.collect_splits_metadata()) + .await?; + info!( + index_uid=%self.params.pipeline_id.index_uid, + source_id=%self.params.pipeline_id.source_id, + "fetched {} splits candidates for merge", + immature_splits.len() + ); + Ok(immature_splits) + } } #[async_trait] @@ -470,7 +497,7 @@ impl Handler for MergePipeline { #[derive(Clone)] pub struct MergePipelineParams { - pub pipeline_id: IndexingPipelineId, + pub pipeline_id: MergePipelineId, pub doc_mapper: Arc, pub indexing_directory: TempDirectory, pub metastore: MetastoreServiceClient, @@ -492,9 +519,9 @@ mod tests { use quickwit_common::ServiceStream; use quickwit_doc_mapper::default_doc_mapper_for_test; use quickwit_metastore::ListSplitsRequestExt; - use quickwit_proto::indexing::IndexingPipelineId; + use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::{MetastoreServiceClient, MockMetastoreService}; - use quickwit_proto::types::{IndexUid, PipelineUid}; + use quickwit_proto::types::{IndexUid, NodeId}; use quickwit_storage::RamStorage; use crate::actors::merge_pipeline::{MergePipeline, MergePipelineParams}; @@ -503,14 +530,15 @@ mod tests { #[tokio::test] async fn test_merge_pipeline_simple() -> anyhow::Result<()> { - let mut mock_metastore = MockMetastoreService::new(); - let index_uid = IndexUid::new_with_random_ulid("test-index"); - let pipeline_id = IndexingPipelineId { + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::for_test("test-index", 0); + let source_id = "test-source".to_string(); + let pipeline_id = MergePipelineId { index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), + source_id, + node_id, }; + let mut mock_metastore = MockMetastoreService::new(); mock_metastore .expect_list_splits() .times(1) @@ -522,7 +550,7 @@ mod tests { vec![quickwit_metastore::SplitState::Published] ); let Bound::Excluded(_) = list_split_query.mature else { - panic!("Expected excluded bound."); + panic!("expected `Bound::Excluded`"); }; true }) @@ -542,7 +570,7 @@ mod tests { merge_io_throughput_limiter_opt: None, event_broker: Default::default(), }; - let pipeline = MergePipeline::new(pipeline_params, universe.spawn_ctx()); + let pipeline = MergePipeline::new(pipeline_params, None, universe.spawn_ctx()); let (_pipeline_mailbox, pipeline_handler) = universe.spawn_builder().spawn(pipeline); let (pipeline_exit_status, pipeline_statistics) = pipeline_handler.quit().await; assert_eq!(pipeline_statistics.generation, 1); diff --git a/quickwit/quickwit-indexing/src/actors/merge_planner.rs b/quickwit/quickwit-indexing/src/actors/merge_planner.rs index 5fcc84f73f..77fb64ad65 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_planner.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_planner.rs @@ -24,7 +24,7 @@ use std::time::Instant; use async_trait::async_trait; use quickwit_actors::{Actor, ActorContext, ActorExitStatus, Handler, Mailbox, QueueCapacity}; use quickwit_metastore::SplitMetadata; -use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::indexing::MergePipelineId; use serde::Serialize; use tantivy::Inventory; use time::OffsetDateTime; @@ -153,15 +153,15 @@ impl MergePlanner { } pub fn new( - pipeline_id: IndexingPipelineId, - published_splits: Vec, + pipeline_id: &MergePipelineId, + immature_splits: Vec, merge_policy: Arc, merge_split_downloader_mailbox: Mailbox, merge_scheduler_service: Mailbox, ) -> MergePlanner { - let published_splits: Vec = published_splits + let immature_splits: Vec = immature_splits .into_iter() - .filter(|split_metadata| belongs_to_pipeline(&pipeline_id, split_metadata)) + .filter(|split_metadata| belongs_to_pipeline(pipeline_id, split_metadata)) .collect(); let mut merge_planner = MergePlanner { known_split_ids: Default::default(), @@ -174,7 +174,7 @@ impl MergePlanner { incarnation_started_at: Instant::now(), }; - merge_planner.record_splits_if_necessary(published_splits); + merge_planner.record_splits_if_necessary(immature_splits); merge_planner } @@ -296,11 +296,11 @@ impl MergePlanner { } } -/// We can merge splits from the same (index_id, source_id, node_id). -fn belongs_to_pipeline(pipeline_id: &IndexingPipelineId, split: &SplitMetadata) -> bool { - pipeline_id.index_uid == split.index_uid +/// We can only merge splits with the same (node_id, index_id, source_id). +fn belongs_to_pipeline(pipeline_id: &MergePipelineId, split: &SplitMetadata) -> bool { + pipeline_id.node_id == split.node_id + && pipeline_id.index_uid == split.index_uid && pipeline_id.source_id == split.source_id - && pipeline_id.node_id == split.node_id } #[derive(Debug)] @@ -325,8 +325,8 @@ mod tests { }; use quickwit_config::IndexingSettings; use quickwit_metastore::{SplitMaturity, SplitMetadata}; - use quickwit_proto::indexing::IndexingPipelineId; - use quickwit_proto::types::{IndexUid, PipelineUid}; + use quickwit_proto::indexing::MergePipelineId; + use quickwit_proto::types::{IndexUid, NodeId}; use time::OffsetDateTime; use crate::actors::MergePlanner; @@ -360,15 +360,13 @@ mod tests { #[tokio::test] async fn test_merge_planner_with_stable_custom_merge_policy() -> anyhow::Result<()> { - let universe = Universe::with_accelerated_time(); + let node_id = NodeId::from("test-node"); let index_uid = IndexUid::new_with_random_ulid("test-index"); - let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = - universe.create_test_mailbox(); - let pipeline_id = IndexingPipelineId { + let source_id = "test-source".to_string(); + let pipeline_id = MergePipelineId { + node_id, index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), + source_id, }; let merge_policy = Arc::new(StableLogMergePolicy::new( StableLogMergePolicyConfig { @@ -379,14 +377,17 @@ mod tests { }, 50_000, )); + let universe = Universe::with_accelerated_time(); + let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = + universe.create_test_mailbox(); + let merge_planner = MergePlanner::new( - pipeline_id, + &pipeline_id, Vec::new(), merge_policy, merge_split_downloader_mailbox, universe.get_or_spawn_one(), ); - let (merge_planner_mailbox, merge_planner_handle) = universe.spawn_builder().spawn(merge_planner); { @@ -447,17 +448,18 @@ mod tests { #[tokio::test] async fn test_merge_planner_spawns_merge_over_existing_splits_on_startup() -> anyhow::Result<()> { + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); + let source_id = "test-source".to_string(); + let pipeline_id = MergePipelineId { + node_id, + index_uid: index_uid.clone(), + source_id, + }; let universe = Universe::with_accelerated_time(); let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Bounded(2)); - let index_uid = IndexUid::new_with_random_ulid("test-index"); - let pipeline_id = IndexingPipelineId { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; let merge_policy_config = ConstWriteAmplificationMergePolicyConfig { merge_factor: 2, max_merge_factor: 2, @@ -468,7 +470,7 @@ mod tests { merge_policy: MergePolicyConfig::ConstWriteAmplification(merge_policy_config), ..Default::default() }; - let pre_existing_splits = vec![ + let immature_splits = vec![ split_metadata_for_test( &index_uid, "a_small", 0, // partition_id 1_000_000, 2, @@ -480,8 +482,8 @@ mod tests { ]; let merge_policy: Arc = merge_policy_from_settings(&indexing_settings); let merge_planner = MergePlanner::new( - pipeline_id, - pre_existing_splits.clone(), + &pipeline_id, + immature_splits.clone(), merge_policy, merge_split_downloader_mailbox, universe.get_or_spawn_one(), @@ -500,7 +502,7 @@ mod tests { // merge. merge_planner_mailbox .ask(NewSplits { - new_splits: pre_existing_splits, + new_splits: immature_splits, }) .await?; @@ -522,19 +524,21 @@ mod tests { #[tokio::test] async fn test_merge_planner_dismiss_splits_from_different_pipeline_id() -> anyhow::Result<()> { + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); + let source_id = "test-source".to_string(); + let pipeline_id = MergePipelineId { + node_id, + index_uid, + source_id, + }; // This test makes sure that the merge planner ignores the splits that do not belong // to the same pipeline let universe = Universe::with_accelerated_time(); let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Bounded(2)); - let index_uid = IndexUid::new_with_random_ulid("test-index"); - let pipeline_id = IndexingPipelineId { - index_uid, - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; + let merge_policy_config = ConstWriteAmplificationMergePolicyConfig { merge_factor: 2, max_merge_factor: 2, @@ -549,7 +553,7 @@ mod tests { // It is different from the index_uid because the index uid has a unique suffix. let other_index_uid = IndexUid::new_with_random_ulid("test-index"); - let pre_existing_splits = vec![ + let immature_splits = vec![ split_metadata_for_test( &other_index_uid, "a_small", @@ -567,8 +571,8 @@ mod tests { ]; let merge_policy: Arc = merge_policy_from_settings(&indexing_settings); let merge_planner = MergePlanner::new( - pipeline_id, - pre_existing_splits.clone(), + &pipeline_id, + immature_splits.clone(), merge_policy, merge_split_downloader_mailbox, universe.get_or_spawn_one(), @@ -588,17 +592,19 @@ mod tests { #[tokio::test] async fn test_merge_planner_inherit_mailbox_with_splits_bug_3847() -> anyhow::Result<()> { + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); + let source_id = "test-source".to_string(); + let pipeline_id = MergePipelineId { + node_id, + index_uid: index_uid.clone(), + source_id, + }; let universe = Universe::with_accelerated_time(); let (merge_split_downloader_mailbox, merge_split_downloader_inbox) = universe .spawn_ctx() .create_mailbox("MergeSplitDownloader", QueueCapacity::Bounded(2)); - let index_uid = IndexUid::new_with_random_ulid("test-index"); - let pipeline_id = IndexingPipelineId { - index_uid: index_uid.clone(), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; + let merge_policy_config = ConstWriteAmplificationMergePolicyConfig { merge_factor: 2, max_merge_factor: 2, @@ -609,8 +615,7 @@ mod tests { merge_policy: MergePolicyConfig::ConstWriteAmplification(merge_policy_config), ..Default::default() }; - - let pre_existing_splits = vec![ + let immature_splits = vec![ split_metadata_for_test( &index_uid, "a_small", 0, // partition_id 1_000_000, 2, @@ -620,16 +625,14 @@ mod tests { 1_000_000, 2, ), ]; - let merge_policy: Arc = merge_policy_from_settings(&indexing_settings); let merge_planner = MergePlanner::new( - pipeline_id, - pre_existing_splits.clone(), + &pipeline_id, + immature_splits.clone(), merge_policy, merge_split_downloader_mailbox, universe.get_or_spawn_one(), ); - // We create a fake old mailbox that contains two new splits and a PlanMerge message from an // old incarnation. This could happen in real life if the merge pipeline failed // right after a `PlanMerge` was pushed to the pipeline. Note that #3847 did not diff --git a/quickwit/quickwit-indexing/src/actors/mod.rs b/quickwit/quickwit-indexing/src/actors/mod.rs index 69df651e3b..9552a426e2 100644 --- a/quickwit/quickwit-indexing/src/actors/mod.rs +++ b/quickwit/quickwit-indexing/src/actors/mod.rs @@ -39,9 +39,7 @@ pub use doc_processor::{DocProcessor, DocProcessorCounters}; pub use index_serializer::IndexSerializer; pub use indexer::{Indexer, IndexerCounters}; pub use indexing_pipeline::{IndexingPipeline, IndexingPipelineParams}; -pub use indexing_service::{ - IndexingService, IndexingServiceCounters, MergePipelineId, INDEXING_DIR_NAME, -}; +pub use indexing_service::{IndexingService, IndexingServiceCounters, INDEXING_DIR_NAME}; pub use merge_executor::{combine_partition_ids, merge_split_attrs, MergeExecutor}; pub use merge_pipeline::MergePipeline; pub use merge_planner::MergePlanner; diff --git a/quickwit/quickwit-indexing/src/actors/packager.rs b/quickwit/quickwit-indexing/src/actors/packager.rs index aaea8ec624..06577544a1 100644 --- a/quickwit/quickwit-indexing/src/actors/packager.rs +++ b/quickwit/quickwit-indexing/src/actors/packager.rs @@ -387,9 +387,8 @@ mod tests { use quickwit_actors::{ObservationType, Universe}; use quickwit_metastore::checkpoint::IndexCheckpointDelta; - use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::search::{deserialize_split_fields, ListFieldsEntryResponse}; - use quickwit_proto::types::{IndexUid, PipelineUid}; + use quickwit_proto::types::{IndexUid, NodeId}; use tantivy::directory::MmapDirectory; use tantivy::schema::{NumericOptions, Schema, Type, FAST, STRING, TEXT}; use tantivy::{doc, DateTime, IndexBuilder, IndexSettings}; @@ -508,20 +507,20 @@ mod tests { } } let index = index_writer.finalize()?; - let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new_with_random_ulid("test-index"), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; + + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); + let source_id = "test-source".to_string(); // TODO: In the future we would like that kind of segment flush to emit a new split, // but this will require work on tantivy. let indexed_split = IndexedSplit { split_attrs: SplitAttrs { + node_id, + index_uid, + source_id, split_id: "test-split".to_string(), partition_id: 17u64, - pipeline_id, num_docs, uncompressed_docs_size_in_bytes: num_docs * 15, time_range: timerange_opt, diff --git a/quickwit/quickwit-indexing/src/actors/uploader.rs b/quickwit/quickwit-indexing/src/actors/uploader.rs index 44e8aaf6d8..f12128bf32 100644 --- a/quickwit/quickwit-indexing/src/actors/uploader.rs +++ b/quickwit/quickwit-indexing/src/actors/uploader.rs @@ -491,9 +491,8 @@ mod tests { use quickwit_common::pubsub::EventSubscriber; use quickwit_common::temp_dir::TempDirectory; use quickwit_metastore::checkpoint::{IndexCheckpointDelta, SourceCheckpointDelta}; - use quickwit_proto::indexing::IndexingPipelineId; use quickwit_proto::metastore::{EmptyResponse, MockMetastoreService}; - use quickwit_proto::types::PipelineUid; + use quickwit_proto::types::NodeId; use quickwit_storage::RamStorage; use tantivy::DateTime; use tokio::sync::oneshot; @@ -505,14 +504,13 @@ mod tests { #[tokio::test] async fn test_uploader_with_sequencer() -> anyhow::Result<()> { quickwit_common::setup_logging_for_tests(); + + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); + let source_id = "test-source".to_string(); + let event_broker = EventBroker::default(); let universe = Universe::new(); - let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new_with_random_ulid("test-index"), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; let (sequencer_mailbox, sequencer_inbox) = universe.create_test_mailbox::>(); let mut mock_metastore = MockMetastoreService::new(); @@ -551,8 +549,10 @@ mod tests { .send_message(PackagedSplitBatch::new( vec![PackagedSplit { split_attrs: SplitAttrs { + node_id, + index_uid, + source_id, partition_id: 3u64, - pipeline_id, time_range: Some( DateTime::from_timestamp_secs(1_628_203_589) ..=DateTime::from_timestamp_secs(1_628_203_640), @@ -587,8 +587,8 @@ mod tests { let publisher_message = match publish_futures.pop().unwrap().await? { SequencerCommand::Discard => panic!( - "Expected `SequencerCommand::Proceed(SplitUpdate)`, got \ - `SequencerCommand::Discard`." + "expected `SequencerCommand::Proceed(SplitUpdate)`, got \ + `SequencerCommand::Discard`" ), SequencerCommand::Proceed(publisher_message) => publisher_message, }; @@ -619,12 +619,10 @@ mod tests { #[tokio::test] async fn test_uploader_with_sequencer_emits_replace() -> anyhow::Result<()> { - let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new_with_random_ulid("test-index"), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); + let source_id = "test-source".to_string(); + let universe = Universe::new(); let (sequencer_mailbox, sequencer_inbox) = universe.create_test_mailbox::>(); @@ -660,9 +658,11 @@ mod tests { let split_scratch_directory_2 = TempDirectory::for_test(); let packaged_split_1 = PackagedSplit { split_attrs: SplitAttrs { + node_id: node_id.clone(), + index_uid: index_uid.clone(), + source_id: source_id.clone(), split_id: "test-split-1".to_string(), partition_id: 3u64, - pipeline_id: pipeline_id.clone(), num_docs: 10, uncompressed_docs_size_in_bytes: 1_000, time_range: Some( @@ -684,9 +684,11 @@ mod tests { }; let package_split_2 = PackagedSplit { split_attrs: SplitAttrs { + node_id, + index_uid, + source_id, split_id: "test-split-2".to_string(), partition_id: 3u64, - pipeline_id, num_docs: 10, uncompressed_docs_size_in_bytes: 1_000, time_range: Some( @@ -768,22 +770,18 @@ mod tests { #[tokio::test] async fn test_uploader_without_sequencer() -> anyhow::Result<()> { - let ref_index_uid: IndexUid = "test-index-no-sequencer:11111111111111111111111111" - .parse() - .unwrap(); - let pipeline_id = IndexingPipelineId { - index_uid: ref_index_uid.clone(), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::for_test("test-index", 0); + let index_uid_clone = index_uid.clone(); + let source_id = "test-source".to_string(); + let universe = Universe::new(); let (publisher_mailbox, publisher_inbox) = universe.create_test_mailbox::(); let mut mock_metastore = MockMetastoreService::new(); mock_metastore .expect_stage_splits() .withf(move |stage_splits_request| -> bool { - stage_splits_request.index_uid() == &ref_index_uid + stage_splits_request.index_uid() == &index_uid_clone }) .times(1) .returning(|_| Ok(EmptyResponse {})); @@ -810,13 +808,15 @@ mod tests { .send_message(PackagedSplitBatch::new( vec![PackagedSplit { split_attrs: SplitAttrs { + node_id, + index_uid, + source_id, + split_id: "test-split".to_string(), partition_id: 3u64, - pipeline_id, time_range: None, uncompressed_docs_size_in_bytes: 1_000, num_docs: 10, replaced_split_ids: Vec::new(), - split_id: "test-split".to_string(), delete_opstamp: 10, num_merge_ops: 0, }, @@ -844,7 +844,7 @@ mod tests { .. } = publisher_inbox.recv_typed_message().await.unwrap(); - assert_eq!(index_uid.index_id, "test-index-no-sequencer"); + assert_eq!(index_uid.index_id, "test-index"); assert_eq!(new_splits.len(), 1); assert!(replaced_split_ids.is_empty()); universe.assert_quit().await; @@ -950,13 +950,11 @@ mod tests { // we need to keep the handle alive. let _subscribe_handle = event_broker.subscribe(report_splits_listener); + let node_id = NodeId::from("test-node"); + let index_uid = IndexUid::new_with_random_ulid("test-index"); + let source_id = "test-source".to_string(); + let universe = Universe::new(); - let pipeline_id = IndexingPipelineId { - index_uid: IndexUid::new_with_random_ulid("test-index"), - source_id: "test-source".to_string(), - node_id: "test-node".to_string(), - pipeline_uid: PipelineUid::default(), - }; let mut mock_metastore = MockMetastoreService::new(); mock_metastore .expect_stage_splits() @@ -986,8 +984,10 @@ mod tests { .send_message(PackagedSplitBatch::new( vec![PackagedSplit { split_attrs: SplitAttrs { + node_id, + index_uid, + source_id, partition_id: 3u64, - pipeline_id, time_range: Some( DateTime::from_timestamp_secs(1_628_203_589) ..=DateTime::from_timestamp_secs(1_628_203_640), diff --git a/quickwit/quickwit-indexing/src/merge_policy/mod.rs b/quickwit/quickwit-indexing/src/merge_policy/mod.rs index b16914f306..1c82903cba 100644 --- a/quickwit/quickwit-indexing/src/merge_policy/mod.rs +++ b/quickwit/quickwit-indexing/src/merge_policy/mod.rs @@ -31,6 +31,7 @@ pub use nop_merge_policy::NopMergePolicy; use quickwit_config::merge_policy_config::MergePolicyConfig; use quickwit_config::IndexingSettings; use quickwit_metastore::{SplitMaturity, SplitMetadata}; +use quickwit_proto::types::SplitId; use serde::Serialize; pub(crate) use stable_log_merge_policy::StableLogMergePolicy; use tantivy::TrackedObject; @@ -86,7 +87,7 @@ impl Deref for MergeTask { pub struct MergeOperation { #[serde(skip_serializing)] pub merge_parent_span: Span, - pub merge_split_id: String, + pub merge_split_id: SplitId, pub splits: Vec, pub operation_type: MergeOperationType, } @@ -210,8 +211,8 @@ pub mod tests { use proptest::prelude::*; use quickwit_actors::Universe; - use quickwit_proto::indexing::IndexingPipelineId; - use quickwit_proto::types::{IndexUid, PipelineUid}; + use quickwit_proto::indexing::{IndexingPipelineId, MergePipelineId}; + use quickwit_proto::types::{IndexUid, NodeId, PipelineUid}; use rand::seq::SliceRandom; use time::OffsetDateTime; @@ -373,13 +374,12 @@ pub mod tests { assert!(!splits.is_empty(), "Split list should not be empty."); let merged_split_id = new_split_id(); let tags = merge_tags(splits); - let pipeline_id = IndexingPipelineId { + let pipeline_id = MergePipelineId { + node_id: NodeId::from("test_node"), index_uid: IndexUid::new_with_random_ulid("test_index"), source_id: "test_source".to_string(), - node_id: "test_node".to_string(), - pipeline_uid: PipelineUid::for_test(0u128), }; - let split_attrs = merge_split_attrs(merged_split_id, &pipeline_id, splits); + let split_attrs = merge_split_attrs(pipeline_id, merged_split_id, splits); create_split_metadata(merge_policy, &split_attrs, tags, 0..0) } @@ -407,11 +407,11 @@ pub mod tests { let pipeline_id = IndexingPipelineId { index_uid: IndexUid::new_with_random_ulid("test-index"), source_id: "test-source".to_string(), - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), pipeline_uid: PipelineUid::default(), }; let merge_planner = MergePlanner::new( - pipeline_id, + &pipeline_id.merge_pipeline_id(), Vec::new(), merge_policy.clone(), merge_task_mailbox, diff --git a/quickwit/quickwit-indexing/src/models/indexed_split.rs b/quickwit/quickwit-indexing/src/models/indexed_split.rs index 6236728d99..5cc5a25e76 100644 --- a/quickwit/quickwit-indexing/src/models/indexed_split.rs +++ b/quickwit/quickwit-indexing/src/models/indexed_split.rs @@ -102,7 +102,9 @@ impl IndexedSplitBuilder { index_builder.single_segment_index_writer(controlled_directory.clone(), 15_000_000)?; Ok(Self { split_attrs: SplitAttrs { - pipeline_id, + node_id: pipeline_id.node_id, + index_uid: pipeline_id.index_uid, + source_id: pipeline_id.source_id, partition_id, split_id, num_docs: 0, @@ -121,10 +123,9 @@ impl IndexedSplitBuilder { #[instrument(name="serialize_split", skip_all, fields( - index_id=%self.split_attrs.pipeline_id.index_uid.index_id, - source_id=%self.split_attrs.pipeline_id.source_id, - node_id=%self.split_attrs.pipeline_id.node_id, - pipeline_uid=%self.split_attrs.pipeline_id.pipeline_uid, + node_id=%self.split_attrs.node_id, + index_uid=%self.split_attrs.index_uid, + source_id=%self.split_attrs.source_id, split_id=%self.split_attrs.split_id, partition_id=%self.split_attrs.partition_id, num_docs=%self.split_attrs.num_docs, diff --git a/quickwit/quickwit-indexing/src/models/indexing_service_message.rs b/quickwit/quickwit-indexing/src/models/indexing_service_message.rs index cd005d235e..4d5d9e95f4 100644 --- a/quickwit/quickwit-indexing/src/models/indexing_service_message.rs +++ b/quickwit/quickwit-indexing/src/models/indexing_service_message.rs @@ -18,22 +18,20 @@ // along with this program. If not, see . use quickwit_config::SourceConfig; -use quickwit_proto::indexing::IndexingPipelineId; -use quickwit_proto::types::PipelineUid; - -use crate::actors::MergePipelineId; +use quickwit_proto::indexing::{IndexingPipelineId, MergePipelineId}; +use quickwit_proto::types::{IndexId, PipelineUid, SourceId}; #[derive(Clone, Debug)] pub struct SpawnPipeline { - pub index_id: String, + pub index_id: IndexId, pub source_config: SourceConfig, pub pipeline_uid: PipelineUid, } #[derive(Clone, Debug)] pub struct ShutdownPipelines { - pub index_id: String, - pub source_id: Option, + pub index_id: IndexId, + pub source_id: Option, // TODO // pub pipeline_ord: Option, } diff --git a/quickwit/quickwit-indexing/src/models/packaged_split.rs b/quickwit/quickwit-indexing/src/models/packaged_split.rs index 078c81a75c..03446bf24b 100644 --- a/quickwit/quickwit-indexing/src/models/packaged_split.rs +++ b/quickwit/quickwit-indexing/src/models/packaged_split.rs @@ -19,6 +19,7 @@ use std::collections::BTreeSet; use std::fmt; +use std::path::PathBuf; use itertools::Itertools; use quickwit_common::temp_dir::TempDirectory; @@ -34,13 +35,13 @@ pub struct PackagedSplit { pub split_attrs: SplitAttrs, pub split_scratch_directory: TempDirectory, pub tags: BTreeSet, - pub split_files: Vec, + pub split_files: Vec, pub hotcache_bytes: Vec, } impl PackagedSplit { pub fn index_uid(&self) -> &IndexUid { - &self.split_attrs.pipeline_id.index_uid + &self.split_attrs.index_uid } pub fn split_id(&self) -> &str { @@ -105,7 +106,7 @@ impl PackagedSplitBatch { } pub fn index_uid(&self) -> IndexUid { - self.splits[0].split_attrs.pipeline_id.index_uid.clone() + self.splits[0].split_attrs.index_uid.clone() } pub fn split_ids(&self) -> Vec { diff --git a/quickwit/quickwit-indexing/src/models/split_attrs.rs b/quickwit/quickwit-indexing/src/models/split_attrs.rs index d36f1a3c34..010dba8153 100644 --- a/quickwit/quickwit-indexing/src/models/split_attrs.rs +++ b/quickwit/quickwit-indexing/src/models/split_attrs.rs @@ -23,20 +23,27 @@ use std::ops::{Range, RangeInclusive}; use std::sync::Arc; use quickwit_metastore::SplitMetadata; -use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::types::{IndexUid, NodeId, SourceId, SplitId}; use tantivy::DateTime; use time::OffsetDateTime; use crate::merge_policy::MergePolicy; pub struct SplitAttrs { + /// ID of the node that produced the split. + pub node_id: NodeId, + // Index UID to which the split belongs. + pub index_uid: IndexUid, + /// Source ID to which the split belongs. + pub source_id: SourceId, + /// Split ID. Joined with the index URI (/), this ID /// should be enough to uniquely identify a split. /// In reality, some information may be implicitly configured /// in the storage resolver: for instance, the Amazon S3 region. - pub split_id: String, + pub split_id: SplitId, - /// Partition to which the split belongs to. + /// Partition to which the split belongs. /// /// Partitions are usually meant to isolate documents based on some field like /// `tenant_id`. For this reason, ideally splits with a different `partition_id` @@ -44,8 +51,6 @@ pub struct SplitAttrs { /// does not hurt correctness however. pub partition_id: u64, - pub pipeline_id: IndexingPipelineId, - /// Number of valid documents in the split. pub num_docs: u64, @@ -92,11 +97,11 @@ pub fn create_split_metadata( let maturity = merge_policy.split_maturity(split_attrs.num_docs as usize, split_attrs.num_merge_ops); SplitMetadata { + node_id: split_attrs.node_id.to_string(), + index_uid: split_attrs.index_uid.clone(), + source_id: split_attrs.source_id.clone(), split_id: split_attrs.split_id.clone(), - index_uid: split_attrs.pipeline_id.index_uid.clone(), partition_id: split_attrs.partition_id, - source_id: split_attrs.pipeline_id.source_id.clone(), - node_id: split_attrs.pipeline_id.node_id.clone(), num_docs: split_attrs.num_docs as usize, time_range: split_attrs .time_range diff --git a/quickwit/quickwit-indexing/src/source/file_source.rs b/quickwit/quickwit-indexing/src/source/file_source.rs index 0faf35b4f8..9ad169f701 100644 --- a/quickwit/quickwit-indexing/src/source/file_source.rs +++ b/quickwit/quickwit-indexing/src/source/file_source.rs @@ -32,7 +32,7 @@ use quickwit_common::uri::Uri; use quickwit_config::FileSourceParams; use quickwit_metastore::checkpoint::PartitionId; use quickwit_proto::metastore::SourceType; -use quickwit_proto::types::Position; +use quickwit_proto::types::{Position, SourceId}; use serde::Serialize; use tokio::io::{AsyncBufReadExt, AsyncRead, AsyncReadExt, BufReader}; use tracing::info; @@ -52,7 +52,7 @@ pub struct FileSourceCounters { } pub struct FileSource { - source_id: String, + source_id: SourceId, params: FileSourceParams, counters: FileSourceCounters, reader: FileSourceReader, diff --git a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs index 7166f1d1fb..a242d2e185 100644 --- a/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs +++ b/quickwit/quickwit-indexing/src/source/gcp_pubsub_source.rs @@ -293,7 +293,7 @@ mod gcp_pubsub_emulator_tests { use google_cloud_pubsub::subscription::SubscriptionConfig; use quickwit_actors::Universe; use quickwit_config::{SourceConfig, SourceInputFormat, SourceParams}; - use quickwit_proto::types::IndexUid; + use quickwit_proto::types::{IndexId, IndexUid}; use serde_json::json; use super::*; @@ -376,7 +376,7 @@ mod gcp_pubsub_emulator_tests { let source_id = source_config.source_id.clone(); let source_loader = quickwit_supported_sources(); - let index_id: String = append_random_suffix("test-gcp-pubsub-source--index"); + let index_id: IndexId = append_random_suffix("test-gcp-pubsub-source--index"); let index_uid = IndexUid::new_with_random_ulid(&index_id); let mut pubsub_messages = Vec::with_capacity(6); diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index 0fa4c06d56..9eb5d6d5b6 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -702,7 +702,7 @@ mod tests { #[tokio::test] async fn test_ingest_source_assign_shards() { let pipeline_id = IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: IndexUid::for_test("test-index", 0), source_id: "test-source".to_string(), pipeline_uid: PipelineUid::default(), @@ -1044,7 +1044,7 @@ mod tests { // - emission of a suggest truncate // - no stream request is emitted let pipeline_id = IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: IndexUid::for_test("test-index", 0), source_id: "test-source".to_string(), pipeline_uid: PipelineUid::default(), @@ -1184,7 +1184,7 @@ mod tests { // - emission of a suggest truncate // - the stream request emitted does not include the EOF shards let pipeline_id = IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: IndexUid::for_test("test-index", 0), source_id: "test-source".to_string(), pipeline_uid: PipelineUid::default(), @@ -1347,7 +1347,7 @@ mod tests { #[tokio::test] async fn test_ingest_source_emit_batches() { let pipeline_id = IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: IndexUid::for_test("test-index", 0), source_id: "test-source".to_string(), pipeline_uid: PipelineUid::default(), @@ -1535,7 +1535,7 @@ mod tests { #[tokio::test] async fn test_ingest_source_emit_batches_shard_not_found() { let pipeline_id = IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: IndexUid::for_test("test-index", 0), source_id: "test-source".to_string(), pipeline_uid: PipelineUid::default(), @@ -1642,7 +1642,7 @@ mod tests { #[tokio::test] async fn test_ingest_source_suggest_truncate() { let pipeline_id = IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: IndexUid::for_test("test-index", 0), source_id: "test-source".to_string(), pipeline_uid: PipelineUid::default(), @@ -1849,7 +1849,7 @@ mod tests { // away. In that case, the ingester should just ignore the assigned shard, as // opposed to fail as the metastore does not let it `acquire` the shard. let pipeline_id = IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: IndexUid::for_test("test-index", 0), source_id: "test-source".to_string(), pipeline_uid: PipelineUid::default(), diff --git a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs index 9f1b30b954..058b952823 100644 --- a/quickwit/quickwit-indexing/src/source/ingest_api_source.rs +++ b/quickwit/quickwit-indexing/src/source/ingest_api_source.rs @@ -29,7 +29,7 @@ use quickwit_ingest::{ }; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpoint}; use quickwit_proto::metastore::SourceType; -use quickwit_proto::types::Position; +use quickwit_proto::types::{Position, SourceId}; use serde::Serialize; use serde_json::Value as JsonValue; use tracing::{error, info}; @@ -58,7 +58,7 @@ pub struct IngestApiSourceCounters { pub struct IngestApiSource { source_runtime: SourceRuntime, - source_id: String, + source_id: SourceId, partition_id: PartitionId, ingest_api_service: Mailbox, counters: IngestApiSourceCounters, @@ -263,7 +263,7 @@ mod tests { }; use quickwit_ingest::{init_ingest_api, CommitType, DocBatchBuilder, IngestRequest}; use quickwit_metastore::checkpoint::SourceCheckpointDelta; - use quickwit_proto::types::IndexUid; + use quickwit_proto::types::{IndexId, IndexUid}; use super::*; use crate::models::RawDocBatch; @@ -271,7 +271,7 @@ mod tests { use crate::source::SourceActor; fn make_ingest_request( - index_id: String, + index_id: IndexId, num_batch: u64, batch_size: usize, commit_type: CommitType, diff --git a/quickwit/quickwit-indexing/src/source/mod.rs b/quickwit/quickwit-indexing/src/source/mod.rs index 1803dc6a7f..dae5d9f505 100644 --- a/quickwit/quickwit-indexing/src/source/mod.rs +++ b/quickwit/quickwit-indexing/src/source/mod.rs @@ -102,7 +102,7 @@ use quickwit_proto::metastore::{ IndexMetadataRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, SourceType, }; -use quickwit_proto::types::{IndexUid, PipelineUid, ShardId}; +use quickwit_proto::types::{IndexUid, NodeIdRef, PipelineUid, ShardId}; use quickwit_storage::StorageResolver; use serde_json::Value as JsonValue; pub use source_factory::{SourceFactory, SourceLoader, TypedSourceFactory}; @@ -146,7 +146,7 @@ pub struct SourceRuntime { } impl SourceRuntime { - pub fn node_id(&self) -> &str { + pub fn node_id(&self) -> &NodeIdRef { &self.pipeline_id.node_id } @@ -547,6 +547,7 @@ mod tests { use quickwit_metastore::checkpoint::IndexCheckpointDelta; use quickwit_metastore::IndexMetadata; use quickwit_proto::metastore::{IndexMetadataResponse, MockMetastoreService}; + use quickwit_proto::types::NodeId; use super::*; @@ -579,7 +580,7 @@ mod tests { SourceRuntime { pipeline_id: IndexingPipelineId { - node_id: "test-node".to_string(), + node_id: NodeId::from("test-node"), index_uid: self.index_uid, source_id: self.source_config.source_id.clone(), pipeline_uid: PipelineUid::for_test(0u128), diff --git a/quickwit/quickwit-indexing/src/source/source_factory.rs b/quickwit/quickwit-indexing/src/source/source_factory.rs index 810cb3bfe6..23cfb1edc9 100644 --- a/quickwit/quickwit-indexing/src/source/source_factory.rs +++ b/quickwit/quickwit-indexing/src/source/source_factory.rs @@ -22,6 +22,7 @@ use std::collections::HashMap; use async_trait::async_trait; use itertools::Itertools; use quickwit_proto::metastore::SourceType; +use quickwit_proto::types::SourceId; use thiserror::Error; use super::Source; @@ -74,7 +75,7 @@ pub enum SourceLoaderError { }, #[error("failed to create source `{source_id}` of type `{source_type}`. Cause: {error:?}")] FailedToCreateSource { - source_id: String, + source_id: SourceId, source_type: SourceType, #[source] error: anyhow::Error, diff --git a/quickwit/quickwit-indexing/src/source/vec_source.rs b/quickwit/quickwit-indexing/src/source/vec_source.rs index 484e480181..85578fe320 100644 --- a/quickwit/quickwit-indexing/src/source/vec_source.rs +++ b/quickwit/quickwit-indexing/src/source/vec_source.rs @@ -25,7 +25,7 @@ use quickwit_actors::{ActorExitStatus, Mailbox}; use quickwit_config::VecSourceParams; use quickwit_metastore::checkpoint::{PartitionId, SourceCheckpointDelta}; use quickwit_proto::metastore::SourceType; -use quickwit_proto::types::Position; +use quickwit_proto::types::{Position, SourceId}; use serde_json::Value as JsonValue; use tracing::info; @@ -34,7 +34,7 @@ use crate::actors::DocProcessor; use crate::source::{Source, SourceContext, SourceRuntime, TypedSourceFactory}; pub struct VecSource { - source_id: String, + source_id: SourceId, source_params: VecSourceParams, next_item_idx: usize, partition: PartitionId, diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 9270d6718e..4c320544bd 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -38,7 +38,7 @@ use quickwit_metastore::{ CreateIndexRequestExt, MetastoreResolver, Split, SplitMetadata, SplitState, }; use quickwit_proto::metastore::{CreateIndexRequest, MetastoreService, MetastoreServiceClient}; -use quickwit_proto::types::{IndexUid, PipelineUid}; +use quickwit_proto::types::{IndexUid, NodeId, PipelineUid, SourceId}; use quickwit_storage::{Storage, StorageResolver}; use serde_json::Value as JsonValue; @@ -51,7 +51,9 @@ use crate::models::{DetachIndexingPipeline, IndexingStatistics, SpawnPipeline}; /// The test index content is entirely in RAM and isolated, /// but the construction of the index involves temporary file directory. pub struct TestSandbox { + node_id: NodeId, index_uid: IndexUid, + source_id: SourceId, indexing_service: Mailbox, doc_mapper: Arc, metastore: MetastoreServiceClient, @@ -76,7 +78,7 @@ impl TestSandbox { indexing_settings_yaml: &str, search_fields: &[&str], ) -> anyhow::Result { - let node_id = append_random_suffix("test-node"); + let node_id = NodeId::new(append_random_suffix("test-node")); let transport = ChannelTransport::default(); let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await @@ -118,7 +120,7 @@ impl TestSandbox { let ingest_api_service = init_ingest_api(&universe, &queues_dir_path, &IngestApiConfig::default()).await?; let indexing_service_actor = IndexingService::new( - node_id.to_string(), + node_id.clone(), temp_dir.path().to_path_buf(), indexer_config, num_blocking_threads, @@ -134,7 +136,9 @@ impl TestSandbox { let (indexing_service, _indexing_service_handle) = universe.spawn_builder().spawn(indexing_service_actor); Ok(TestSandbox { + node_id, index_uid, + source_id: INGEST_API_SOURCE_ID.to_string(), indexing_service, doc_mapper, metastore, @@ -214,11 +218,21 @@ impl TestSandbox { self.doc_mapper.clone() } + /// Returns the node ID. + pub fn node_id(&self) -> NodeId { + self.node_id.clone() + } + /// Returns the index UID. pub fn index_uid(&self) -> IndexUid { self.index_uid.clone() } + /// Returns the source ID. + pub fn source_id(&self) -> SourceId { + self.source_id.clone() + } + /// Returns the underlying universe. pub fn universe(&self) -> &Universe { &self.universe diff --git a/quickwit/quickwit-ingest/src/doc_batch.rs b/quickwit/quickwit-ingest/src/doc_batch.rs index b912cf31c9..f5423194b3 100644 --- a/quickwit/quickwit-ingest/src/doc_batch.rs +++ b/quickwit/quickwit-ingest/src/doc_batch.rs @@ -19,6 +19,7 @@ use bytes::buf::Writer; use bytes::{Buf, BufMut, Bytes, BytesMut}; +use quickwit_proto::types::IndexId; use serde::Serialize; use crate::DocBatch; @@ -96,14 +97,14 @@ where T: Buf + Default /// Builds DocBatch from individual commands pub struct DocBatchBuilder { - index_id: String, + index_id: IndexId, doc_buffer: BytesMut, doc_lengths: Vec, } impl DocBatchBuilder { /// Creates a new batch builder for the given index name. - pub fn new(index_id: String) -> Self { + pub fn new(index_id: IndexId) -> Self { Self { index_id, doc_buffer: BytesMut::new(), @@ -113,7 +114,7 @@ impl DocBatchBuilder { /// Creates a new batch builder for the given index name with some pre-allocated capacity for /// the internal doc buffer. - pub fn with_capacity(index_id: String, capacity: usize) -> Self { + pub fn with_capacity(index_id: IndexId, capacity: usize) -> Self { Self { index_id, doc_buffer: BytesMut::with_capacity(capacity), @@ -171,7 +172,7 @@ impl DocBatchBuilder { /// A wrapper around batch builder that can add a Serialize structs pub struct JsonDocBatchBuilder { - index_id: String, + index_id: IndexId, doc_buffer: Writer, doc_lengths: Vec, } diff --git a/quickwit/quickwit-ingest/src/error.rs b/quickwit/quickwit-ingest/src/error.rs index 3d4fa42caf..52a1f9b6fd 100644 --- a/quickwit/quickwit-ingest/src/error.rs +++ b/quickwit/quickwit-ingest/src/error.rs @@ -24,6 +24,7 @@ use quickwit_actors::AskError; use quickwit_common::tower::BufferError; pub(crate) use quickwit_proto::error::{grpc_error_to_grpc_status, grpc_status_to_service_error}; use quickwit_proto::ingest::IngestV2Error; +use quickwit_proto::types::IndexId; use quickwit_proto::{tonic, GrpcServiceError, ServiceError, ServiceErrorCode}; use serde::{Deserialize, Serialize}; @@ -32,9 +33,9 @@ pub enum IngestServiceError { #[error("data corruption: {0}")] Corruption(String), #[error("index `{index_id}` already exists")] - IndexAlreadyExists { index_id: String }, + IndexAlreadyExists { index_id: IndexId }, #[error("index `{index_id}` not found")] - IndexNotFound { index_id: String }, + IndexNotFound { index_id: IndexId }, #[error("an internal error occurred: {0}")] Internal(String), #[error("invalid position: {0}")] diff --git a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs index 8d6c183547..1f73539311 100644 --- a/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs +++ b/quickwit/quickwit-integration-tests/src/test_utils/cluster_sandbox.rs @@ -35,6 +35,7 @@ use quickwit_config::service::QuickwitService; use quickwit_config::NodeConfig; use quickwit_metastore::{MetastoreResolver, SplitState}; use quickwit_proto::opentelemetry::proto::collector::trace::v1::trace_service_client::TraceServiceClient; +use quickwit_proto::types::NodeId; use quickwit_rest_client::models::IngestSource; use quickwit_rest_client::rest_client::{ CommitType, QuickwitClient, QuickwitClientBuilder, DEFAULT_BASE_URL, @@ -414,8 +415,8 @@ pub fn build_node_configs( let mut config = NodeConfig::for_test(); config.enabled_services = node_services.clone(); config.cluster_id = cluster_id.clone(); - config.node_id = format!("test-node-{node_idx}"); - config.data_dir_path = root_data_dir.join(&config.node_id); + config.node_id = NodeId::new(format!("test-node-{node_idx}")); + config.data_dir_path = root_data_dir.join(config.node_id.as_str()); config.metastore_uri = QuickwitUri::from_str(&format!("ram:///{unique_dir_name}/metastore")).unwrap(); config.default_index_root_uri = diff --git a/quickwit/quickwit-jaeger/src/integration_tests.rs b/quickwit/quickwit-jaeger/src/integration_tests.rs index d1c5ba4de3..54386cd397 100644 --- a/quickwit/quickwit-jaeger/src/integration_tests.rs +++ b/quickwit/quickwit-jaeger/src/integration_tests.rs @@ -46,7 +46,7 @@ use quickwit_proto::metastore::{ }; use quickwit_proto::opentelemetry::proto::collector::trace::v1::trace_service_server::TraceService; use quickwit_proto::opentelemetry::proto::collector::trace::v1::ExportTraceServiceRequest; -use quickwit_proto::types::{IndexUid, PipelineUid}; +use quickwit_proto::types::{IndexUid, NodeId, PipelineUid}; use quickwit_search::{ start_searcher_service, SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool, @@ -340,7 +340,7 @@ async fn indexer_for_test( ) -> (Mailbox, ActorHandle) { let indexer_config = IndexerConfig::for_test().unwrap(); let indexing_service = IndexingService::new( - "test-node".to_string(), + NodeId::from("test-node"), data_dir_path.to_path_buf(), indexer_config, 1, diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index bd6edffa8f..50fc78b721 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -38,9 +38,9 @@ use quickwit_indexing::actors::{ use quickwit_indexing::merge_policy::merge_policy_from_settings; use quickwit_indexing::{IndexingSplitStore, PublisherType, SplitsUpdateMailbox}; use quickwit_metastore::IndexMetadataResponseExt; -use quickwit_proto::indexing::IndexingPipelineId; +use quickwit_proto::indexing::MergePipelineId; use quickwit_proto::metastore::{IndexMetadataRequest, MetastoreService, MetastoreServiceClient}; -use quickwit_proto::types::{IndexUid, PipelineUid}; +use quickwit_proto::types::{IndexUid, NodeId}; use quickwit_search::SearchJobPlacer; use quickwit_storage::Storage; use serde::Serialize; @@ -156,7 +156,7 @@ impl DeleteTaskPipeline { pub async fn spawn_pipeline(&mut self, ctx: &ActorContext) -> anyhow::Result<()> { info!( - index_id=%self.index_uid.index_id, + index_uid=%self.index_uid, root_dir=%self.delete_service_task_dir.to_str().unwrap(), "spawning delete tasks pipeline", ); @@ -193,10 +193,9 @@ impl DeleteTaskPipeline { let tag_fields = doc_mapper.tag_named_fields()?; let packager = Packager::new("MergePackager", tag_fields, uploader_mailbox); let (packager_mailbox, packager_supervisor_handler) = ctx.spawn_actor().supervise(packager); - let index_pipeline_id = IndexingPipelineId { + let pipeline_id = MergePipelineId { + node_id: NodeId::from("unknown"), index_uid: self.index_uid.clone(), - node_id: "unknown".to_string(), - pipeline_uid: PipelineUid::new(), source_id: "unknown".to_string(), }; @@ -206,7 +205,7 @@ impl DeleteTaskPipeline { .clone() .set_component("split_downloader_delete"); let delete_executor = MergeExecutor::new( - index_pipeline_id, + pipeline_id, self.metastore.clone(), doc_mapper.clone(), delete_executor_io_controls, diff --git a/quickwit/quickwit-lambda/src/indexer/ingest/helpers.rs b/quickwit/quickwit-lambda/src/indexer/ingest/helpers.rs index 55704650a9..f35931af59 100644 --- a/quickwit/quickwit-lambda/src/indexer/ingest/helpers.rs +++ b/quickwit/quickwit-lambda/src/indexer/ingest/helpers.rs @@ -36,9 +36,7 @@ use quickwit_config::{ SourceInputFormat, SourceParams, TransformConfig, }; use quickwit_index_management::IndexService; -use quickwit_indexing::actors::{ - IndexingService, MergePipeline, MergePipelineId, MergeSchedulerService, -}; +use quickwit_indexing::actors::{IndexingService, MergePipeline, MergeSchedulerService}; use quickwit_indexing::models::{DetachIndexingPipeline, DetachMergePipeline, SpawnPipeline}; use quickwit_indexing::IndexingPipeline; use quickwit_ingest::IngesterPool; @@ -51,7 +49,7 @@ use quickwit_proto::metastore::{ CreateIndexRequest, IndexMetadataRequest, MetastoreError, MetastoreService, MetastoreServiceClient, ResetSourceCheckpointRequest, }; -use quickwit_proto::types::{NodeId, PipelineUid}; +use quickwit_proto::types::PipelineUid; use quickwit_search::SearchJobPlacer; use quickwit_storage::StorageResolver; use quickwit_telemetry::payload::{QuickwitFeature, QuickwitTelemetryInfo, TelemetryEvent}; @@ -72,7 +70,7 @@ pub(super) async fn create_empty_cluster( services: &[QuickwitService], ) -> anyhow::Result { let self_node = ClusterMember { - node_id: NodeId::new(config.node_id.clone()), + node_id: config.node_id.clone(), generation_id: quickwit_cluster::GenerationId::now(), is_ready: false, enabled_services: HashSet::from_iter(services.to_owned()), @@ -274,7 +272,7 @@ pub(super) async fn spawn_pipelines( .await?; let merge_pipeline_handle = indexing_server_mailbox .ask_for_res(DetachMergePipeline { - pipeline_id: MergePipelineId::from(&pipeline_id), + pipeline_id: pipeline_id.merge_pipeline_id(), }) .await?; let indexing_pipeline_handle = indexing_server_mailbox diff --git a/quickwit/quickwit-metastore/migrations/postgresql/19_add-split-node-id-field.down.sql b/quickwit/quickwit-metastore/migrations/postgresql/19_add-split-node-id-field.down.sql new file mode 100644 index 0000000000..ad1a00f3a4 --- /dev/null +++ b/quickwit/quickwit-metastore/migrations/postgresql/19_add-split-node-id-field.down.sql @@ -0,0 +1,4 @@ +DROP INDEX IF EXISTS splits_node_id_idx; + +ALTER TABLE splits + DROP IF EXISTS COLUMN node_id; diff --git a/quickwit/quickwit-metastore/migrations/postgresql/19_add-split-node-id-field.up.sql b/quickwit/quickwit-metastore/migrations/postgresql/19_add-split-node-id-field.up.sql new file mode 100644 index 0000000000..4fb953976a --- /dev/null +++ b/quickwit/quickwit-metastore/migrations/postgresql/19_add-split-node-id-field.up.sql @@ -0,0 +1,12 @@ +ALTER TABLE splits + ADD COLUMN node_id VARCHAR(253); + +UPDATE + splits +SET + node_id = splits.split_metadata_json::json ->> 'node_id'; + +ALTER TABLE splits + ALTER COLUMN node_id SET NOT NULL; + +CREATE INDEX IF NOT EXISTS splits_node_id_idx ON splits USING HASH (node_id); diff --git a/quickwit/quickwit-metastore/src/checkpoint.rs b/quickwit/quickwit-metastore/src/checkpoint.rs index 159e0c7a39..00cf497c16 100644 --- a/quickwit/quickwit-metastore/src/checkpoint.rs +++ b/quickwit/quickwit-metastore/src/checkpoint.rs @@ -372,7 +372,7 @@ impl fmt::Debug for SourceCheckpoint { /// we are not trying to add documents to the index that were already indexed. #[derive(Clone, PartialEq, Eq, Serialize, Deserialize)] pub struct IndexCheckpointDelta { - pub source_id: String, + pub source_id: SourceId, pub source_delta: SourceCheckpointDelta, } diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs index 5e846d5a93..567ae67918 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs @@ -705,12 +705,18 @@ fn split_query_predicate(split: &&Split, query: &ListSplitsQuery) -> bool { Bound::Unbounded => {} } - if let Some(range) = split.split_metadata.time_range.as_ref() { + if let Some(range) = &split.split_metadata.time_range { if !query.time_range.overlaps_with(range.clone()) { return false; } } + if let Some(node_id) = &query.node_id { + if split.split_metadata.node_id != *node_id { + return false; + } + } + true } diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index 4cb6819ab0..3ca762f606 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -41,7 +41,7 @@ use quickwit_proto::metastore::{ MetastoreResult, MetastoreService, MetastoreServiceClient, MetastoreServiceStream, PublishSplitsRequest, StageSplitsRequest, UpdateIndexRequest, }; -use quickwit_proto::types::{IndexUid, SplitId}; +use quickwit_proto::types::{IndexUid, NodeId, SplitId}; use time::OffsetDateTime; use crate::checkpoint::IndexCheckpointDelta; @@ -556,9 +556,12 @@ impl ListSplitsResponseExt for ListSplitsResponse { #[derive(Debug, Clone, serde::Serialize, serde::Deserialize)] /// A query builder for listing splits within the metastore. pub struct ListSplitsQuery { - /// A non-empty list of index UIDs to get splits from. + /// A non-empty list of index UIDs for which to fetch the splits. pub index_uids: Vec, + /// A specific node ID to filter by. + pub node_id: Option, + /// The maximum number of splits to retrieve. pub limit: Option, @@ -597,6 +600,7 @@ impl ListSplitsQuery { pub fn for_index(index_uid: IndexUid) -> Self { Self { index_uids: vec![index_uid], + node_id: None, limit: None, offset: None, split_states: Vec::new(), @@ -621,6 +625,7 @@ impl ListSplitsQuery { } Ok(Self { index_uids, + node_id: None, limit: None, offset: None, split_states: Vec::new(), @@ -634,6 +639,12 @@ impl ListSplitsQuery { }) } + /// Selects splits produced by the specified node. + pub fn with_node_id(mut self, node_id: NodeId) -> Self { + self.node_id = Some(node_id); + self + } + /// Sets the maximum number of splits to retrieve. pub fn with_limit(mut self, n: usize) -> Self { self.limit = Some(n); @@ -646,130 +657,130 @@ impl ListSplitsQuery { self } - /// Select splits which have the given split state. + /// Selects splits which have the given split state. pub fn with_split_state(mut self, state: SplitState) -> Self { self.split_states.push(state); self } - /// Select splits which have the any of the following split state. + /// Selects splits which have the any of the following split state. pub fn with_split_states(mut self, states: impl AsRef<[SplitState]>) -> Self { self.split_states.extend_from_slice(states.as_ref()); self } - /// Select splits which match the given tag filter. + /// Selects splits which match the given tag filter. pub fn with_tags_filter(mut self, tags: TagFilterAst) -> Self { self.tags = Some(tags); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than or equal to* the provided value. pub fn with_time_range_end_lte(mut self, v: i64) -> Self { self.time_range.end = Bound::Included(v); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than* the provided value. pub fn with_time_range_end_lt(mut self, v: i64) -> Self { self.time_range.end = Bound::Excluded(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than or equal to* the provided value. pub fn with_time_range_start_gte(mut self, v: i64) -> Self { self.time_range.start = Bound::Included(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than* the provided value. pub fn with_time_range_start_gt(mut self, v: i64) -> Self { self.time_range.start = Bound::Excluded(v); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than or equal to* the provided value. pub fn with_delete_opstamp_lte(mut self, v: u64) -> Self { self.delete_opstamp.end = Bound::Included(v); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than* the provided value. pub fn with_delete_opstamp_lt(mut self, v: u64) -> Self { self.delete_opstamp.end = Bound::Excluded(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than or equal to* the provided value. pub fn with_delete_opstamp_gte(mut self, v: u64) -> Self { self.delete_opstamp.start = Bound::Included(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than* the provided value. pub fn with_delete_opstamp_gt(mut self, v: u64) -> Self { self.delete_opstamp.start = Bound::Excluded(v); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than or equal to* the provided value. pub fn with_update_timestamp_lte(mut self, v: i64) -> Self { self.update_timestamp.end = Bound::Included(v); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than* the provided value. pub fn with_update_timestamp_lt(mut self, v: i64) -> Self { self.update_timestamp.end = Bound::Excluded(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than or equal to* the provided value. pub fn with_update_timestamp_gte(mut self, v: i64) -> Self { self.update_timestamp.start = Bound::Included(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than* the provided value. pub fn with_update_timestamp_gt(mut self, v: i64) -> Self { self.update_timestamp.start = Bound::Excluded(v); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than or equal to* the provided value. pub fn with_create_timestamp_lte(mut self, v: i64) -> Self { self.create_timestamp.end = Bound::Included(v); self } - /// Set the field's lower bound to match values that are + /// Sets the field's lower bound to match values that are /// *less than* the provided value. pub fn with_create_timestamp_lt(mut self, v: i64) -> Self { self.create_timestamp.end = Bound::Excluded(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than or equal to* the provided value. pub fn with_create_timestamp_gte(mut self, v: i64) -> Self { self.create_timestamp.start = Bound::Included(v); self } - /// Set the field's upper bound to match values that are + /// Sets the field's upper bound to match values that are /// *greater than* the provided value. pub fn with_create_timestamp_gt(mut self, v: i64) -> Self { self.create_timestamp.start = Bound::Excluded(v); diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index 207e8a8437..89e887c5e6 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -559,24 +559,24 @@ impl MetastoreService for PostgresqlMetastore { &mut self, request: StageSplitsRequest, ) -> MetastoreResult { - let split_metadata_list = request.deserialize_splits_metadata()?; + let splits_metadata = request.deserialize_splits_metadata()?; + + if splits_metadata.is_empty() { + return Ok(Default::default()); + } let index_uid: IndexUid = request.index_uid().clone(); - let mut split_ids = Vec::with_capacity(split_metadata_list.len()); - let mut time_range_start_list = Vec::with_capacity(split_metadata_list.len()); - let mut time_range_end_list = Vec::with_capacity(split_metadata_list.len()); - let mut tags_list = Vec::with_capacity(split_metadata_list.len()); - let mut split_metadata_json_list = Vec::with_capacity(split_metadata_list.len()); - let mut delete_opstamps = Vec::with_capacity(split_metadata_list.len()); - let mut maturity_timestamps = Vec::with_capacity(split_metadata_list.len()); - - for split_metadata in split_metadata_list { - let split_metadata_json = serde_json::to_string(&split_metadata).map_err(|error| { - MetastoreError::JsonSerializeError { - struct_name: "SplitMetadata".to_string(), - message: error.to_string(), - } - })?; - split_metadata_json_list.push(split_metadata_json); + let mut split_ids = Vec::with_capacity(splits_metadata.len()); + let mut time_range_start_list = Vec::with_capacity(splits_metadata.len()); + let mut time_range_end_list = Vec::with_capacity(splits_metadata.len()); + let mut tags_list = Vec::with_capacity(splits_metadata.len()); + let mut splits_metadata_json = Vec::with_capacity(splits_metadata.len()); + let mut delete_opstamps = Vec::with_capacity(splits_metadata.len()); + let mut maturity_timestamps = Vec::with_capacity(splits_metadata.len()); + let mut node_ids = Vec::with_capacity(splits_metadata.len()); + + for split_metadata in splits_metadata { + let split_metadata_json = serde_utils::to_json_str(&split_metadata)?; + splits_metadata_json.push(split_metadata_json); let time_range_start = split_metadata .time_range @@ -592,13 +592,15 @@ impl MetastoreService for PostgresqlMetastore { tags_list.push(sqlx::types::Json(tags)); split_ids.push(split_metadata.split_id); delete_opstamps.push(split_metadata.delete_opstamp as i64); + node_ids.push(split_metadata.node_id); } tracing::Span::current().record("split_ids", format!("{split_ids:?}")); + // TODO: Remove transaction. run_with_tx!(self.connection_pool, tx, { let upserted_split_ids: Vec = sqlx::query_scalar(r#" INSERT INTO splits - (split_id, time_range_start, time_range_end, tags, split_metadata_json, delete_opstamp, maturity_timestamp, split_state, index_uid) + (split_id, time_range_start, time_range_end, tags, split_metadata_json, delete_opstamp, maturity_timestamp, split_state, index_uid, node_id) SELECT split_id, time_range_start, @@ -607,11 +609,12 @@ impl MetastoreService for PostgresqlMetastore { split_metadata_json, delete_opstamp, to_timestamp(maturity_timestamp), - $8 as split_state, - $9 as index_uid + $9 as split_state, + $10 as index_uid, + node_id FROM - UNNEST($1, $2, $3, $4, $5, $6, $7) - AS staged_splits (split_id, time_range_start, time_range_end, tags_json, split_metadata_json, delete_opstamp, maturity_timestamp) + UNNEST($1, $2, $3, $4, $5, $6, $7, $8) + AS staged_splits (split_id, time_range_start, time_range_end, tags_json, split_metadata_json, delete_opstamp, maturity_timestamp, node_id) ON CONFLICT(split_id) DO UPDATE SET time_range_start = excluded.time_range_start, @@ -621,6 +624,7 @@ impl MetastoreService for PostgresqlMetastore { delete_opstamp = excluded.delete_opstamp, maturity_timestamp = excluded.maturity_timestamp, index_uid = excluded.index_uid, + node_id = excluded.node_id, update_timestamp = CURRENT_TIMESTAMP, create_timestamp = CURRENT_TIMESTAMP WHERE splits.split_id = excluded.split_id AND splits.split_state = 'Staged' @@ -630,9 +634,10 @@ impl MetastoreService for PostgresqlMetastore { .bind(time_range_start_list) .bind(time_range_end_list) .bind(tags_list) - .bind(split_metadata_json_list) + .bind(splits_metadata_json) .bind(delete_opstamps) .bind(maturity_timestamps) + .bind(&node_ids) .bind(SplitState::Staged.as_str()) .bind(&index_uid) .fetch_all(tx.as_mut()) @@ -651,7 +656,7 @@ impl MetastoreService for PostgresqlMetastore { return Err(MetastoreError::FailedPrecondition { entity, message }); } info!( - index_id=%index_uid.index_id, + %index_uid, "staged `{}` splits successfully", split_ids.len() ); Ok(EmptyResponse {}) @@ -818,7 +823,7 @@ impl MetastoreService for PostgresqlMetastore { return Err(MetastoreError::FailedPrecondition { entity, message }); } info!( - index_id=%index_uid.index_id, + %index_uid, "published {} splits and marked {} for deletion successfully", num_published_splits, num_marked_splits ); @@ -936,14 +941,14 @@ impl MetastoreService for PostgresqlMetastore { })); } info!( - index_id=%index_uid.index_id, + %index_uid, "Marked {} splits for deletion, among which {} were newly marked.", split_ids.len() - not_found_split_ids.len(), num_marked_splits ); if !not_found_split_ids.is_empty() { warn!( - index_id=%index_uid.index_id, + %index_uid, split_ids=?PrettySample::new(¬_found_split_ids, 5), "{} splits were not found and could not be marked for deletion.", not_found_split_ids.len() @@ -1028,11 +1033,11 @@ impl MetastoreService for PostgresqlMetastore { }; return Err(MetastoreError::FailedPrecondition { entity, message }); } - info!(index_id=%index_uid.index_id, "Deleted {} splits from index.", num_deleted_splits); + info!(%index_uid, "deleted {} splits from index", num_deleted_splits); if !not_found_split_ids.is_empty() { warn!( - index_id=%index_uid.index_id, + %index_uid, split_ids=?PrettySample::new(¬_found_split_ids, 5), "{} splits were not found and could not be deleted.", not_found_split_ids.len() diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/model.rs b/quickwit/quickwit-metastore/src/metastore/postgres/model.rs index 341ffcbe9f..1b10190715 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/model.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/model.rs @@ -24,7 +24,7 @@ use std::str::FromStr; use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{DeleteQuery, DeleteTask, MetastoreError, MetastoreResult}; -use quickwit_proto::types::{IndexUid, ShardId, SourceId}; +use quickwit_proto::types::{IndexId, IndexUid, ShardId, SourceId, SplitId}; use sea_query::{Iden, Write}; use tracing::error; @@ -38,7 +38,7 @@ pub(super) struct PgIndex { #[sqlx(try_from = "String")] pub index_uid: IndexUid, /// Index ID. The index ID is used to resolve user queries. - pub index_id: String, + pub index_id: IndexId, // A JSON string containing all of the IndexMetadata. pub index_metadata_json: String, /// Timestamp for tracking when the split was created. @@ -81,6 +81,7 @@ pub enum Splits { Tags, SplitMetadataJson, IndexUid, + NodeId, DeleteOpstamp, } @@ -96,7 +97,7 @@ impl Iden for ToTimestampFunc { #[derive(sqlx::FromRow)] pub(super) struct PgSplit { /// Split ID. - pub split_id: String, + pub split_id: SplitId, /// The state of the split. With `update_timestamp`, this is the only mutable attribute of the /// split. pub split_state: String, diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/utils.rs b/quickwit/quickwit-metastore/src/metastore/postgres/utils.rs index 1f972749c4..63d1f6722f 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/utils.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/utils.rs @@ -96,9 +96,11 @@ pub(super) fn append_range_filters( pub(super) fn append_query_filters(sql: &mut SelectStatement, query: &ListSplitsQuery) { // Note: `ListSplitsQuery` builder enforces a non empty `index_uids` list. - sql.cond_where( - Expr::col(Splits::IndexUid).is_in(query.index_uids.iter().map(|val| val.to_string())), - ); + sql.cond_where(Expr::col(Splits::IndexUid).is_in(&query.index_uids)); + + if let Some(node_id) = &query.node_id { + sql.cond_where(Expr::col(Splits::NodeId).eq(node_id)); + }; if !query.split_states.is_empty() { sql.cond_where( diff --git a/quickwit/quickwit-metastore/src/split_metadata.rs b/quickwit/quickwit-metastore/src/split_metadata.rs index 44c5f57685..7221dfd49a 100644 --- a/quickwit/quickwit-metastore/src/split_metadata.rs +++ b/quickwit/quickwit-metastore/src/split_metadata.rs @@ -25,7 +25,7 @@ use std::str::FromStr; use std::time::Duration; use bytesize::ByteSize; -use quickwit_proto::types::IndexUid; +use quickwit_proto::types::{IndexUid, SourceId, SplitId}; use serde::{Deserialize, Serialize}; use serde_with::{serde_as, DurationMilliSeconds}; use time::OffsetDateTime; @@ -68,7 +68,7 @@ pub struct SplitMetadata { /// should be enough to uniquely identify a split. /// In reality, some information may be implicitly configured /// in the storage resolver: for instance, the Amazon S3 region. - pub split_id: String, + pub split_id: SplitId, /// Id of the index this split belongs to. pub index_uid: IndexUid, @@ -82,7 +82,7 @@ pub struct SplitMetadata { pub partition_id: u64, /// Source ID. - pub source_id: String, + pub source_id: SourceId, /// Node ID. pub node_id: String, @@ -183,10 +183,10 @@ impl fmt::Debug for SplitMetadata { impl SplitMetadata { /// Creates a new instance of split metadata. pub fn new( - split_id: String, + split_id: SplitId, index_uid: IndexUid, partition_id: u64, - source_id: String, + source_id: SourceId, node_id: String, ) -> Self { Self { @@ -220,7 +220,7 @@ impl SplitMetadata { #[cfg(any(test, feature = "testsuite"))] /// Returns an instance of `SplitMetadata` for testing. - pub fn for_test(split_id: String) -> SplitMetadata { + pub fn for_test(split_id: SplitId) -> SplitMetadata { SplitMetadata { split_id, ..Default::default() @@ -245,7 +245,7 @@ impl SplitMetadata { #[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] pub struct SplitInfo { /// The split ID. - pub split_id: String, + pub split_id: SplitId, /// The number of documents in the split. pub num_docs: usize, /// The sum of the sizes of the original JSON payloads in bytes. diff --git a/quickwit/quickwit-metastore/src/split_metadata_version.rs b/quickwit/quickwit-metastore/src/split_metadata_version.rs index 476eabb95f..d60b8dddcd 100644 --- a/quickwit/quickwit-metastore/src/split_metadata_version.rs +++ b/quickwit/quickwit-metastore/src/split_metadata_version.rs @@ -20,7 +20,7 @@ use std::collections::BTreeSet; use std::ops::{Range, RangeInclusive}; -use quickwit_proto::types::IndexUid; +use quickwit_proto::types::{IndexUid, SplitId}; use serde::{Deserialize, Serialize}; use crate::split_metadata::{utc_now_timestamp, SplitMaturity}; @@ -32,7 +32,7 @@ pub(crate) struct SplitMetadataV0_8 { /// should be enough to uniquely identify a split. /// In reality, some information may be implicitly configured /// in the storage resolver: for instance, the Amazon S3 region. - pub split_id: String, + pub split_id: SplitId, /// Uid of the index this split belongs to. #[schema(value_type = String)] diff --git a/quickwit/quickwit-metastore/src/tests/list_splits.rs b/quickwit/quickwit-metastore/src/tests/list_splits.rs index 8863caf373..a97bdcebac 100644 --- a/quickwit/quickwit-metastore/src/tests/list_splits.rs +++ b/quickwit/quickwit-metastore/src/tests/list_splits.rs @@ -28,7 +28,7 @@ use quickwit_proto::metastore::{ CreateIndexRequest, ListSplitsRequest, ListStaleSplitsRequest, MarkSplitsForDeletionRequest, PublishSplitsRequest, StageSplitsRequest, }; -use quickwit_proto::types::{IndexUid, SplitId}; +use quickwit_proto::types::{IndexUid, NodeId, SplitId}; use time::OffsetDateTime; use tokio::time::sleep; use tracing::info; @@ -898,6 +898,68 @@ pub async fn test_metastore_list_splits() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); + let index_id = append_random_suffix("test-list-splits-by-node-id"); + let index_uri = format!("ram:///indexes/{index_id}"); + let index_config = IndexConfig::for_test(&index_id, &index_uri); + + let create_index_request = CreateIndexRequest::try_from_index_config(&index_config).unwrap(); + let index_uid: IndexUid = metastore + .create_index(create_index_request) + .await + .unwrap() + .index_uid + .unwrap(); + + let split_id_1 = format!("{index_id}--split-1"); + let split_metadata_1 = SplitMetadata { + split_id: split_id_1.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 20, + node_id: "test-node-1".to_string(), + ..Default::default() + }; + let split_id_2 = format!("{index_id}--split-2"); + let split_metadata_2 = SplitMetadata { + split_id: split_id_2.clone(), + index_uid: index_uid.clone(), + create_timestamp: current_timestamp, + delete_opstamp: 10, + node_id: "test-node-2".to_string(), + ..Default::default() + }; + let stage_splits_request = StageSplitsRequest::try_from_splits_metadata( + index_uid.clone(), + vec![split_metadata_1.clone(), split_metadata_2.clone()], + ) + .unwrap(); + + metastore.stage_splits(stage_splits_request).await.unwrap(); + + let list_splits_query = + ListSplitsQuery::for_index(index_uid.clone()).with_node_id(NodeId::from("test-node-1")); + let list_splits_request = + ListSplitsRequest::try_from_list_splits_query(&list_splits_query).unwrap(); + + let splits = metastore + .list_splits(list_splits_request) + .await + .unwrap() + .collect_splits() + .await + .unwrap(); + + assert_eq!(splits.len(), 1); + assert_eq!(splits[0].split_metadata.split_id, split_id_1); + assert_eq!(splits[0].split_metadata.node_id, "test-node-1"); +} + pub async fn test_metastore_list_stale_splits< MetastoreToTest: MetastoreServiceExt + DefaultForTest, >() { diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index ecabfe513a..a8d6820548 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -312,6 +312,12 @@ macro_rules! metastore_test_suite { $crate::tests::list_splits::test_metastore_list_splits::<$metastore_type>().await; } + #[tokio::test] + async fn test_metastore_list_splits_by_node() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::list_splits::test_metastore_list_splits_by_node_id::<$metastore_type>().await; + } + #[tokio::test] async fn test_metastore_split_update_timestamp() { let _ = tracing_subscriber::fmt::try_init(); diff --git a/quickwit/quickwit-metastore/src/tests/split.rs b/quickwit/quickwit-metastore/src/tests/split.rs index 55632bbb86..e1590d827e 100644 --- a/quickwit/quickwit-metastore/src/tests/split.rs +++ b/quickwit/quickwit-metastore/src/tests/split.rs @@ -35,7 +35,7 @@ use tracing::{error, info}; use super::DefaultForTest; use crate::checkpoint::{IndexCheckpointDelta, PartitionId, SourceCheckpointDelta}; use crate::metastore::MetastoreServiceStreamSplitsExt; -use crate::tests::{cleanup_index, collect_split_ids}; +use crate::tests::cleanup_index; use crate::{ CreateIndexRequestExt, IndexMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, MetastoreServiceExt, SplitMetadata, SplitState, StageSplitsRequestExt, @@ -1536,6 +1536,7 @@ pub async fn test_metastore_stage_splits> for IndexingError { } } +/// Uniquely identifies an indexing pipeline. There can be multiple indexing pipelines per +/// source `(index_uid, source_id)` running simultaneously on an indexer. #[derive(Clone, Debug, Hash, Eq, PartialEq)] pub struct IndexingPipelineId { - pub node_id: String, + pub node_id: NodeId, pub index_uid: IndexUid, pub source_id: SourceId, pub pipeline_uid: PipelineUid, } +impl IndexingPipelineId { + pub fn merge_pipeline_id(&self) -> MergePipelineId { + MergePipelineId { + node_id: self.node_id.clone(), + index_uid: self.index_uid.clone(), + source_id: self.source_id.clone(), + } + } +} + impl Display for IndexingPipelineId { fn fmt(&self, f: &mut Formatter) -> fmt::Result { write!(f, "{}:{}", self.index_uid, &self.source_id) } } +/// Uniquely identifies a merge pipeline. There exists at most one merge pipeline per +/// `(index_uid, source_id)` running on indexer at any given time fed by one or more indexing +/// pipelines. +#[derive(Clone, Debug, Hash, Eq, PartialEq)] +pub struct MergePipelineId { + pub node_id: NodeId, + pub index_uid: IndexUid, + pub source_id: SourceId, +} + +impl Display for MergePipelineId { + fn fmt(&self, f: &mut Formatter) -> fmt::Result { + write!(f, "merge:{}:{}", self.index_uid, &self.source_id) + } +} + impl Display for IndexingTask { fn fmt(&self, f: &mut Formatter) -> fmt::Result { write!(f, "{}:{}", self.index_uid(), &self.source_id) diff --git a/quickwit/quickwit-proto/src/metastore/mod.rs b/quickwit/quickwit-proto/src/metastore/mod.rs index b4e5e06cbd..d065791ef1 100644 --- a/quickwit/quickwit-proto/src/metastore/mod.rs +++ b/quickwit/quickwit-proto/src/metastore/mod.rs @@ -73,7 +73,7 @@ pub enum EntityKind { /// A set of splits. Splits { /// Split IDs. - split_ids: Vec, + split_ids: Vec, }, /// An index template. IndexTemplate { diff --git a/quickwit/quickwit-proto/src/types/index_uid.rs b/quickwit/quickwit-proto/src/types/index_uid.rs index a437038cda..9382261227 100644 --- a/quickwit/quickwit-proto/src/types/index_uid.rs +++ b/quickwit/quickwit-proto/src/types/index_uid.rs @@ -232,9 +232,16 @@ impl PartialEq<(&'static str, u128)> for IndexUid { } } +#[cfg(feature = "postgres")] +impl From for sea_query::Value { + fn from(index_uid: IndexUid) -> Self { + index_uid.to_string().into() + } +} + #[cfg(feature = "postgres")] impl From<&IndexUid> for sea_query::Value { - fn from(val: &IndexUid) -> Self { - val.to_string().into() + fn from(index_uid: &IndexUid) -> Self { + index_uid.to_string().into() } } diff --git a/quickwit/quickwit-proto/src/types/mod.rs b/quickwit/quickwit-proto/src/types/mod.rs index 18e034e628..998239db68 100644 --- a/quickwit/quickwit-proto/src/types/mod.rs +++ b/quickwit/quickwit-proto/src/types/mod.rs @@ -264,6 +264,13 @@ impl ToOwned for NodeIdRef { } } +#[cfg(feature = "postgres")] +impl From<&NodeId> for sea_query::Value { + fn from(node_id: &NodeId) -> Self { + node_id.to_string().into() + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/quickwit/quickwit-search/src/collector.rs b/quickwit/quickwit-search/src/collector.rs index ae33f328b3..24b2b159f2 100644 --- a/quickwit/quickwit-search/src/collector.rs +++ b/quickwit/quickwit-search/src/collector.rs @@ -28,6 +28,7 @@ use quickwit_proto::search::{ LeafSearchResponse, PartialHit, SearchRequest, SortByValue, SortOrder, SortValue, SplitSearchError, }; +use quickwit_proto::types::SplitId; use serde::Deserialize; use tantivy::aggregation::agg_req::{get_fast_field_names, Aggregations}; use tantivy::aggregation::intermediate_agg_result::IntermediateAggregationResults; @@ -536,7 +537,7 @@ pub(crate) struct SegmentPartialHit { impl SegmentPartialHit { pub fn into_partial_hit( self, - split_id: String, + split_id: SplitId, segment_ord: SegmentOrdinal, first: &SortingFieldExtractorComponent, second: &Option, @@ -711,7 +712,7 @@ impl QuickwitIncrementalAggregations { sort_value: Some(SortValue::I64(timestamp)), }), sort_value2: None, - split_id: String::new(), + split_id: SplitId::new(), segment_ord: 0, doc_id: 0, }); @@ -753,7 +754,7 @@ impl QuickwitIncrementalAggregations { /// the query. #[derive(Clone)] pub(crate) struct QuickwitCollector { - pub split_id: String, + pub split_id: SplitId, pub start_offset: usize, pub max_hits: usize, pub sort_by: SortByPair, @@ -1044,7 +1045,7 @@ pub(crate) fn sort_by_from_request(search_request: &SearchRequest) -> SortByPair /// Builds the QuickwitCollector, in function of the information that was requested by the user. pub(crate) fn make_collector_for_split( - split_id: String, + split_id: SplitId, search_request: &SearchRequest, aggregation_limits: AggregationLimits, ) -> crate::Result { @@ -1075,7 +1076,7 @@ pub(crate) fn make_merge_collector( }; let sort_by = sort_by_from_request(search_request); Ok(QuickwitCollector { - split_id: String::default(), + split_id: SplitId::default(), start_offset: search_request.start_offset as usize, max_hits: search_request.max_hits as usize, sort_by, diff --git a/quickwit/quickwit-search/src/leaf_cache.rs b/quickwit/quickwit-search/src/leaf_cache.rs index 548d5744e1..f94f2d6462 100644 --- a/quickwit/quickwit-search/src/leaf_cache.rs +++ b/quickwit/quickwit-search/src/leaf_cache.rs @@ -23,6 +23,7 @@ use prost::Message; use quickwit_proto::search::{ CountHits, LeafSearchResponse, SearchRequest, SplitIdAndFooterOffsets, }; +use quickwit_proto::types::SplitId; use quickwit_storage::{MemorySizedCache, OwnedBytes}; /// A cache to memoize `leaf_search_single_split` results. @@ -82,7 +83,7 @@ impl LeafSearchCache { #[derive(Debug, Hash, PartialEq, Eq)] struct CacheKey { /// The split this entry refers to - split_id: String, + split_id: SplitId, /// The request this matches. The timerange of the request was removed. request: SearchRequest, /// The effective time range of the request, that is, the intersection of the timerange diff --git a/quickwit/quickwit-search/src/list_fields.rs b/quickwit/quickwit-search/src/list_fields.rs index d6030dbe8f..a45cc19b7f 100644 --- a/quickwit/quickwit-search/src/list_fields.rs +++ b/quickwit/quickwit-search/src/list_fields.rs @@ -34,7 +34,7 @@ use quickwit_proto::search::{ deserialize_split_fields, LeafListFieldsRequest, ListFields, ListFieldsEntryResponse, ListFieldsRequest, ListFieldsResponse, SplitIdAndFooterOffsets, }; -use quickwit_proto::types::IndexUid; +use quickwit_proto::types::{IndexId, IndexUid}; use quickwit_storage::Storage; use crate::leaf::open_split_bundle; @@ -45,7 +45,7 @@ use crate::{list_relevant_splits, resolve_index_patterns, ClusterClient, SearchE /// Get the list of splits for the request which we need to scan. pub async fn get_fields_from_split<'a>( searcher_context: &SearcherContext, - index_id: String, + index_id: IndexId, split_and_footer_offsets: &'a SplitIdAndFooterOffsets, index_storage: Arc, ) -> anyhow::Result + Send>> { @@ -230,7 +230,7 @@ fn matches_pattern(field_pattern: &str, field_name: &str) -> bool { /// `leaf` step of list fields. pub async fn leaf_list_fields( - index_id: String, + index_id: IndexId, index_storage: Arc, searcher_context: &SearcherContext, split_ids: &[SplitIdAndFooterOffsets], @@ -276,7 +276,7 @@ pub async fn leaf_list_fields( #[derive(Clone, Debug)] pub struct IndexMetasForLeafSearch { /// Index id. - pub index_id: String, + pub index_id: IndexId, /// Index URI. pub index_uri: Uri, } diff --git a/quickwit/quickwit-search/src/list_fields_cache.rs b/quickwit/quickwit-search/src/list_fields_cache.rs index 1d11ff8307..0ab04c584f 100644 --- a/quickwit/quickwit-search/src/list_fields_cache.rs +++ b/quickwit/quickwit-search/src/list_fields_cache.rs @@ -20,6 +20,7 @@ use quickwit_proto::search::{ deserialize_split_fields, serialize_split_fields, ListFields, SplitIdAndFooterOffsets, }; +use quickwit_proto::types::SplitId; use quickwit_storage::{MemorySizedCache, OwnedBytes}; /// A cache to memoize `leaf_search_single_split` results. @@ -57,7 +58,7 @@ impl ListFieldsCache { #[derive(Debug, Hash, PartialEq, Eq)] struct CacheKey { /// The split this entry refers to - split_id: String, + split_id: SplitId, } impl CacheKey { diff --git a/quickwit/quickwit-search/src/top_k_collector.rs b/quickwit/quickwit-search/src/top_k_collector.rs index 898badd13d..41f55025d2 100644 --- a/quickwit/quickwit-search/src/top_k_collector.rs +++ b/quickwit/quickwit-search/src/top_k_collector.rs @@ -23,6 +23,7 @@ use std::marker::PhantomData; use quickwit_common::binary_heap::TopK; use quickwit_proto::search::{PartialHit, SortOrder}; +use quickwit_proto::types::SplitId; use tantivy::{DocId, Score}; use crate::collector::{ @@ -192,7 +193,7 @@ impl< } pub fn specialized_top_k_segment_collector( - split_id: String, + split_id: SplitId, score_extractor: SortingFieldExtractorPair, leaf_max_hits: usize, segment_ord: u32, @@ -546,7 +547,7 @@ struct SpecializedSegmentTopKCollector< V2: Copy + PartialEq + Eq + PartialOrd + Ord + IntoOptionU64 + Debug + MinValue, const REVERSE_DOCID: bool, > { - split_id: String, + split_id: SplitId, hit_fetcher: SpecSortingFieldExtractor, top_k_hits: TopKComputer>, segment_ord: u32, @@ -559,7 +560,7 @@ impl< > SpecializedSegmentTopKCollector { pub fn new( - split_id: String, + split_id: SplitId, score_extractor: SortingFieldExtractorPair, leaf_max_hits: usize, segment_ord: u32, @@ -614,7 +615,7 @@ impl< /// Quickwit collector working at the scale of the segment. pub(crate) struct GenericQuickwitSegmentTopKCollector { - split_id: String, + split_id: SplitId, score_extractor: SortingFieldExtractorPair, // PartialHits in this heap don't contain a split_id yet. top_k_hits: TopK, @@ -628,7 +629,7 @@ pub(crate) struct GenericQuickwitSegmentTopKCollector { impl GenericQuickwitSegmentTopKCollector { pub fn new( - split_id: String, + split_id: SplitId, score_extractor: SortingFieldExtractorPair, leaf_max_hits: usize, segment_ord: u32, diff --git a/quickwit/quickwit-serve/src/delete_task_api/handler.rs b/quickwit/quickwit-serve/src/delete_task_api/handler.rs index 64f02bf824..9542b2807e 100644 --- a/quickwit/quickwit-serve/src/delete_task_api/handler.rs +++ b/quickwit/quickwit-serve/src/delete_task_api/handler.rs @@ -25,7 +25,7 @@ use quickwit_proto::metastore::{ MetastoreService, MetastoreServiceClient, }; use quickwit_proto::search::SearchRequest; -use quickwit_proto::types::IndexUid; +use quickwit_proto::types::{IndexId, IndexUid}; use quickwit_query::query_ast::{query_ast_from_user_text, QueryAst}; use serde::Deserialize; use warp::{Filter, Rejection}; @@ -94,7 +94,7 @@ pub fn get_delete_tasks_handler( // `DeleteTaskService`. This is ensured by requiring a `Mailbox` in // `get_delete_tasks_handler` and consequently we get the mailbox in `get_delete_tasks` signature. pub async fn get_delete_tasks( - index_id: String, + index_id: IndexId, mut metastore: MetastoreServiceClient, ) -> MetastoreResult> { let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); @@ -140,7 +140,7 @@ pub fn post_delete_tasks_handler( /// This operation will not be immediately executed, instead it will be added to a queue /// and cleaned up in the near future. pub async fn post_delete_request( - index_id: String, + index_id: IndexId, delete_request: DeleteQueryRequest, mut metastore: MetastoreServiceClient, ) -> Result { diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index a64ff91854..dc1dcf14e5 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -39,7 +39,7 @@ use quickwit_proto::metastore::{ MetastoreService, MetastoreServiceClient, ResetSourceCheckpointRequest, ToggleSourceRequest, UpdateIndexRequest, }; -use quickwit_proto::types::IndexUid; +use quickwit_proto::types::{IndexId, IndexUid, SourceId}; use quickwit_query::query_ast::{query_ast_from_user_text, QueryAst}; use serde::de::DeserializeOwned; use serde::{Deserialize, Serialize}; @@ -126,7 +126,7 @@ fn get_index_metadata_handler( } async fn get_index_metadata( - index_id: String, + index_id: IndexId, mut metastore: MetastoreServiceClient, ) -> MetastoreResult { info!(index_id = %index_id, "get-index-metadata"); @@ -164,7 +164,7 @@ fn list_indexes_metadata_handler( /// Describes an index with its main information and statistics. #[derive(Serialize, Deserialize, utoipa::ToSchema)] struct IndexStats { - pub index_id: String, + pub index_id: IndexId, #[schema(value_type = String)] pub index_uri: Uri, pub num_published_splits: usize, @@ -190,7 +190,7 @@ struct IndexStats { /// Describes an index. async fn describe_index( - index_id: String, + index_id: IndexId, mut metastore: MetastoreServiceClient, ) -> MetastoreResult { let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); @@ -317,7 +317,7 @@ pub struct ListSplitsResponse { /// Get splits. async fn list_splits( - index_id: String, + index_id: IndexId, list_split_query: ListSplitsQueryParams, mut metastore: MetastoreServiceClient, ) -> MetastoreResult { @@ -394,7 +394,7 @@ struct SplitsForDeletion { )] /// Marks splits for deletion. async fn mark_splits_for_deletion( - index_id: String, + index_id: IndexId, splits_for_deletion: SplitsForDeletion, mut metastore: MetastoreServiceClient, ) -> MetastoreResult<()> { @@ -567,7 +567,7 @@ fn update_index_handler( /// configuration are replaced by the values specified in the request. In particular, omitting an /// optional field like `retention_policy` will delete the associated configuration. async fn update_index( - index_id: String, + index_id: IndexId, request: IndexUpdates, mut metastore: MetastoreServiceClient, ) -> Result { @@ -613,7 +613,7 @@ fn clear_index_handler( /// Removes all of the data (splits, queued document) associated with the index, but keeps the index /// configuration. (See also, `delete-index`). async fn clear_index( - index_id: String, + index_id: IndexId, mut index_service: IndexService, ) -> Result<(), IndexServiceError> { info!(index_id = %index_id, "clear-index"); @@ -654,7 +654,7 @@ fn delete_index_handler( )] /// Deletes index. async fn delete_index( - index_id: String, + index_id: IndexId, delete_index_query_param: DeleteIndexQueryParam, mut index_service: IndexService, ) -> Result, IndexServiceError> { @@ -694,7 +694,7 @@ fn create_source_handler( )] /// Creates Source. async fn create_source( - index_id: String, + index_id: IndexId, config_format: ConfigFormat, source_config_bytes: Bytes, mut index_service: IndexService, @@ -732,8 +732,8 @@ fn get_source_handler( } async fn get_source( - index_id: String, - source_id: String, + index_id: IndexId, + source_id: SourceId, mut metastore: MetastoreServiceClient, ) -> MetastoreResult { info!(index_id = %index_id, source_id = %source_id, "get-source"); @@ -778,8 +778,8 @@ fn reset_source_checkpoint_handler( )] /// Resets source checkpoint. async fn reset_source_checkpoint( - index_id: String, - source_id: String, + index_id: IndexId, + source_id: SourceId, mut metastore: MetastoreServiceClient, ) -> MetastoreResult<()> { let index_metadata_resquest = IndexMetadataRequest::for_index_id(index_id.to_string()); @@ -832,8 +832,8 @@ struct ToggleSource { )] /// Toggles source. async fn toggle_source( - index_id: String, - source_id: String, + index_id: IndexId, + source_id: SourceId, toggle_source: ToggleSource, mut metastore: MetastoreServiceClient, ) -> Result<(), IndexServiceError> { @@ -884,8 +884,8 @@ fn delete_source_handler( )] /// Deletes source. async fn delete_source( - index_id: String, - source_id: String, + index_id: IndexId, + source_id: SourceId, mut metastore: MetastoreServiceClient, ) -> Result<(), IndexServiceError> { info!(index_id = %index_id, source_id = %source_id, "delete-source"); diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index 895a095f0f..d01f2fa424 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -208,7 +208,7 @@ fn convert_ingest_response_v2( )] /// Ingest documents async fn ingest( - index_id: String, + index_id: IndexId, body: Body, ingest_options: IngestOptions, mut ingest_service: IngestServiceClient, @@ -258,7 +258,7 @@ fn tail_filter() -> impl Filter + Clone )] /// Returns the last few ingested documents. async fn tail_endpoint( - index_id: String, + index_id: IndexId, mut ingest_service: IngestServiceClient, ) -> Result { let fetch_response = ingest_service.tail(TailRequest { index_id }).await?; diff --git a/quickwit/quickwit-serve/src/otlp_api/rest_handler.rs b/quickwit/quickwit-serve/src/otlp_api/rest_handler.rs index 45c33b8893..940504e820 100644 --- a/quickwit/quickwit-serve/src/otlp_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/otlp_api/rest_handler.rs @@ -29,6 +29,7 @@ use quickwit_proto::opentelemetry::proto::collector::trace::v1::trace_service_se use quickwit_proto::opentelemetry::proto::collector::trace::v1::{ ExportTraceServiceRequest, ExportTraceServiceResponse, }; +use quickwit_proto::types::IndexId; use quickwit_proto::{tonic, ServiceError, ServiceErrorCode}; use serde::{self, Serialize}; use tracing::error; @@ -139,7 +140,7 @@ impl ServiceError for OtlpApiError { async fn otlp_ingest_logs( otlp_logs_service: OtlpGrpcLogsService, - _index_id: String, // <- TODO: use index ID when gRPC service supports it. + _index_id: IndexId, // <- TODO: use index ID when gRPC service supports it. body: Bytes, ) -> Result { // TODO: use index ID. @@ -154,7 +155,7 @@ async fn otlp_ingest_logs( async fn otlp_ingest_traces( otlp_traces_service: OtlpGrpcTracesService, - _index_id: String, // <- TODO: use index ID when gRPC service supports it. + _index_id: IndexId, // <- TODO: use index ID when gRPC service supports it. body: Bytes, ) -> Result { let export_traces_request: ExportTraceServiceRequest = prost::Message::decode(&body[..]) diff --git a/quickwit/quickwit-serve/src/search_api/rest_handler.rs b/quickwit/quickwit-serve/src/search_api/rest_handler.rs index 1b6eddafee..a51fb439e8 100644 --- a/quickwit/quickwit-serve/src/search_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/search_api/rest_handler.rs @@ -26,6 +26,7 @@ use hyper::HeaderMap; use percent_encoding::percent_decode_str; use quickwit_config::validate_index_id_pattern; use quickwit_proto::search::{CountHits, OutputFormat, SortField, SortOrder}; +use quickwit_proto::types::IndexId; use quickwit_proto::ServiceError; use quickwit_query::query_ast::query_ast_from_user_text; use quickwit_search::{SearchError, SearchResponseRest, SearchService}; @@ -431,7 +432,7 @@ struct SearchStreamRequestQueryString { } async fn search_stream_endpoint( - index_id: String, + index_id: IndexId, search_request: SearchStreamRequestQueryString, search_service: &dyn SearchService, ) -> Result { @@ -498,7 +499,7 @@ fn make_streaming_reply(result: Result) -> impl Reply } async fn search_stream( - index_id: String, + index_id: IndexId, request: SearchStreamRequestQueryString, search_service: Arc, ) -> impl warp::Reply {