From 12bf2395d1b3c44b66a141b37c2f60a3cdf7a9a0 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:56 +0100 Subject: [PATCH 01/11] Add docs --- docs/reference/rest-api.md | 50 +++++++++++++++++++++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/docs/reference/rest-api.md b/docs/reference/rest-api.md index 896084d4b24..a74af3f66b2 100644 --- a/docs/reference/rest-api.md +++ b/docs/reference/rest-api.md @@ -606,10 +606,58 @@ Create source by posting a source config JSON payload. | `version** | `String` | Config format version, put your current Quickwit version. | _required_ | | `source_id` | `String` | Source ID. See ID [validation rules](../configuration/source-config.md). | _required_ | | `source_type` | `String` | Source type: `kafka`, `kinesis` or `pulsar`. | _required_ | -| `num_pipelines` | `usize` | Number of running indexing pipelines per node for this source. | 1 | +| `num_pipelines` | `usize` | Number of running indexing pipelines per node for this source. | `1` | +| `transform` | `object` | A [VRL](https://vector.dev/docs/reference/vrl/) transformation applied to incoming documents, as defined in [source config docs](../configuration/source-config.md#transform-parameters). | `null` | | `params` | `object` | Source parameters as defined in [source config docs](../configuration/source-config.md). | _required_ | +**Payload Example** + +curl -XPOST http://localhost:7280/api/v1/indexes/my-index/sources --data @source_config.json -H "Content-Type: application/json" + +```json title="source_config.json +{ + "version": "0.8", + "source_id": "kafka-source", + "source_type": "kafka", + "params": { + "topic": "quickwit-fts-staging", + "client_params": { + "bootstrap.servers": "kafka-quickwit-server:9092" + } + } +} +``` + +#### Response + +The response is the created source config, and the content type is `application/json; charset=UTF-8.` + +### Update a source + +``` +PUT api/v1/indexes//sources/ +``` + +Update a source by posting a source config JSON payload. + +#### PUT payload + +| Variable | Type | Description | Default value | +|-------------------|----------|----------------------------------------------------------------------------------------|---------------| +| `version** | `String` | Config format version, put your current Quickwit version. | _required_ | +| `source_id` | `String` | Source ID, must be the same source as in the request URL. | _required_ | +| `source_type` | `String` | Source type: `kafka`, `kinesis` or `pulsar`. Cannot be updated. | _required_ | +| `num_pipelines` | `usize` | Number of running indexing pipelines per node for this source. | `1` | +| `transform` | `object` | A [VRL](https://vector.dev/docs/reference/vrl/) transformation applied to incoming documents, as defined in [source config docs](../configuration/source-config.md#transform-parameters). | `null` | +| `params` | `object` | Source parameters as defined in [source config docs](../configuration/source-config.md). | _required_ | + +:::warning + +While updating `num_pipelines` and `transform` is generally safe and reversible, updating `params` has consequences specific to the source type and might have side effects such as loosing the source's checkpoints. Perform such updates with great care. + +::: + **Payload Example** curl -XPOST http://localhost:7280/api/v1/indexes/my-index/sources --data @source_config.json -H "Content-Type: application/json" From ba40f21de0e543ef3c8d7cddf3e1df114f14d79c Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:56 +0100 Subject: [PATCH 02/11] Refactor index API into resources --- .../src/index_api/index_resource.rs | 435 ++++++++ quickwit/quickwit-serve/src/index_api/mod.rs | 10 +- .../src/index_api/rest_handler.rs | 934 +----------------- .../src/index_api/source_resource.rs | 335 +++++++ .../src/index_api/split_resource.rs | 212 ++++ 5 files changed, 1029 insertions(+), 897 deletions(-) create mode 100644 quickwit/quickwit-serve/src/index_api/index_resource.rs create mode 100644 quickwit/quickwit-serve/src/index_api/source_resource.rs create mode 100644 quickwit/quickwit-serve/src/index_api/split_resource.rs diff --git a/quickwit/quickwit-serve/src/index_api/index_resource.rs b/quickwit/quickwit-serve/src/index_api/index_resource.rs new file mode 100644 index 00000000000..650b0d2294f --- /dev/null +++ b/quickwit/quickwit-serve/src/index_api/index_resource.rs @@ -0,0 +1,435 @@ +// Copyright (C) 2024 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use std::sync::Arc; + +use bytes::Bytes; +use quickwit_common::uri::Uri; +use quickwit_config::{ + load_index_config_update, validate_index_id_pattern, ConfigFormat, NodeConfig, +}; +use quickwit_index_management::{IndexService, IndexServiceError}; +use quickwit_metastore::{ + IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsQuery, + ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, Split, SplitInfo, SplitState, + UpdateIndexRequestExt, +}; +use quickwit_proto::metastore::{ + IndexMetadataRequest, ListIndexesMetadataRequest, ListSplitsRequest, MetastoreError, + MetastoreResult, MetastoreService, MetastoreServiceClient, UpdateIndexRequest, +}; +use quickwit_proto::types::IndexId; +use serde::{Deserialize, Serialize}; +use tracing::info; +use warp::{Filter, Rejection}; + +use super::rest_handler::log_failure; +use crate::format::{extract_config_format, extract_format_from_qs}; +use crate::rest_api_response::into_rest_api_response; +use crate::simple_list::from_simple_list; +use crate::with_arg; + +pub fn get_index_metadata_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String) + .and(warp::get()) + .and(with_arg(metastore)) + .then(get_index_metadata) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +pub async fn get_index_metadata( + index_id: IndexId, + metastore: MetastoreServiceClient, +) -> MetastoreResult { + info!(index_id = %index_id, "get-index-metadata"); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; + Ok(index_metadata) +} + +/// This struct represents the QueryString passed to +/// the rest API to filter indexes. +#[derive(Debug, Clone, Deserialize, Serialize, utoipa::IntoParams, utoipa::ToSchema, Default)] +#[into_params(parameter_in = Query)] +pub struct ListIndexesQueryParams { + #[serde(deserialize_with = "from_simple_list")] + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub index_id_patterns: Option>, +} + +pub fn list_indexes_metadata_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes") + .and(warp::get()) + .and(serde_qs::warp::query(serde_qs::Config::default())) + .and(with_arg(metastore)) + .then(list_indexes_metadata) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +/// Describes an index with its main information and statistics. +#[derive(Serialize, Deserialize, utoipa::ToSchema)] +pub struct IndexStats { + #[schema(value_type = String)] + pub index_id: IndexId, + #[schema(value_type = String)] + pub index_uri: Uri, + pub num_published_splits: usize, + pub size_published_splits: u64, + pub num_published_docs: u64, + pub size_published_docs_uncompressed: u64, + pub timestamp_field_name: Option, + pub min_timestamp: Option, + pub max_timestamp: Option, +} + +#[utoipa::path( + get, + tag = "Indexes", + path = "/indexes/{index_id}/describe", + responses( + (status = 200, description = "Successfully fetched stats about Index.", body = IndexStats) + ), + params( + ("index_id" = String, Path, description = "The index ID to describe."), + ) +)] + +/// Describes an index. +pub async fn describe_index( + index_id: IndexId, + metastore: MetastoreServiceClient, +) -> MetastoreResult { + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_metadata = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; + let query = ListSplitsQuery::for_index(index_metadata.index_uid.clone()); + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(&query)?; + let splits = metastore + .list_splits(list_splits_request) + .await? + .collect_splits() + .await?; + let published_splits: Vec = splits + .into_iter() + .filter(|split| split.split_state == SplitState::Published) + .collect(); + let mut total_num_docs = 0; + let mut total_num_bytes = 0; + let mut total_uncompressed_num_bytes = 0; + let mut min_timestamp: Option = None; + let mut max_timestamp: Option = None; + + for split in &published_splits { + total_num_docs += split.split_metadata.num_docs as u64; + total_num_bytes += split.split_metadata.footer_offsets.end; + total_uncompressed_num_bytes += split.split_metadata.uncompressed_docs_size_in_bytes; + + if let Some(time_range) = &split.split_metadata.time_range { + min_timestamp = min_timestamp + .min(Some(*time_range.start())) + .or(Some(*time_range.start())); + max_timestamp = max_timestamp + .max(Some(*time_range.end())) + .or(Some(*time_range.end())); + } + } + + let index_config = index_metadata.into_index_config(); + let index_stats = IndexStats { + index_id, + index_uri: index_config.index_uri.clone(), + num_published_splits: published_splits.len(), + size_published_splits: total_num_bytes, + num_published_docs: total_num_docs, + size_published_docs_uncompressed: total_uncompressed_num_bytes, + timestamp_field_name: index_config.doc_mapping.timestamp_field, + min_timestamp, + max_timestamp, + }; + + Ok(index_stats) +} + +pub fn describe_index_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "describe") + .and(warp::get()) + .and(with_arg(metastore)) + .then(describe_index) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + get, + tag = "Indexes", + path = "/indexes", + responses( + // We return `VersionedIndexMetadata` as it's the serialized model view. + (status = 200, description = "Successfully fetched all indexes.", body = [VersionedIndexMetadata]) + ), + params( + ListIndexesQueryParams, + ("index_id_patterns" = String, Path, description = "The index ID pattern to retrieve indexes for."), + ) +)] +/// Gets indexes metadata. +pub async fn list_indexes_metadata( + list_indexes_params: ListIndexesQueryParams, + metastore: MetastoreServiceClient, +) -> MetastoreResult> { + let list_indexes_metata_request = + if let Some(index_id_patterns) = list_indexes_params.index_id_patterns { + for index_id_pattern in &index_id_patterns { + validate_index_id_pattern(index_id_pattern, true).map_err(|error| { + MetastoreError::InvalidArgument { + message: error.to_string(), + } + })?; + } + ListIndexesMetadataRequest { index_id_patterns } + } else { + ListIndexesMetadataRequest::all() + }; + metastore + .list_indexes_metadata(list_indexes_metata_request) + .await? + .deserialize_indexes_metadata() + .await +} + +#[derive(Deserialize, utoipa::IntoParams, utoipa::ToSchema)] +#[into_params(parameter_in = Query)] +pub struct CreateIndexQueryParams { + #[serde(default)] + overwrite: bool, +} + +pub fn create_index_handler( + index_service: IndexService, + node_config: Arc, +) -> impl Filter + Clone { + warp::path!("indexes") + .and(warp::post()) + .and(serde_qs::warp::query(serde_qs::Config::default())) + .and(extract_config_format()) + .and(warp::body::content_length_limit(1024 * 1024)) + .and(warp::filters::body::bytes()) + .and(with_arg(index_service)) + .and(with_arg(node_config)) + .then(create_index) + .map(log_failure("failed to create index")) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + post, + tag = "Indexes", + path = "/indexes", + request_body = VersionedIndexConfig, + responses( + // We return `VersionedIndexMetadata` as it's the serialized model view. + (status = 200, description = "Successfully created index.", body = VersionedIndexMetadata) + ), + params( + CreateIndexQueryParams, + ) +)] +/// Creates index. +pub async fn create_index( + create_index_query_params: CreateIndexQueryParams, + config_format: ConfigFormat, + index_config_bytes: Bytes, + mut index_service: IndexService, + node_config: Arc, +) -> Result { + let index_config = quickwit_config::load_index_config_from_user_config( + config_format, + &index_config_bytes, + &node_config.default_index_root_uri, + ) + .map_err(IndexServiceError::InvalidConfig)?; + info!(index_id = %index_config.index_id, overwrite = create_index_query_params.overwrite, "create-index"); + index_service + .create_index(index_config, create_index_query_params.overwrite) + .await +} + +pub fn update_index_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String) + .and(warp::put()) + .and(extract_config_format()) + .and(warp::body::content_length_limit(1024 * 1024)) + .and(warp::filters::body::bytes()) + .and(with_arg(metastore)) + .then(update_index) + .map(log_failure("failed to update index")) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + put, + tag = "Indexes", + path = "/indexes/{index_id}", + request_body = VersionedIndexConfig, + responses( + (status = 200, description = "Successfully updated the index configuration.", body = VersionedIndexMetadata) + ), + params( + ("index_id" = String, Path, description = "The index ID to update."), + ) +)] +/// Updates an existing index. +/// +/// This endpoint follows PUT semantics, which means that all the fields of the +/// current configuration are replaced by the values specified in this request +/// or the associated defaults. In particular, if the field is optional (e.g. +/// `retention_policy`), omitting it will delete the associated configuration. +/// If the new configuration file contains updates that cannot be applied, the +/// request fails, and none of the updates are applied. +pub async fn update_index( + target_index_id: IndexId, + config_format: ConfigFormat, + index_config_bytes: Bytes, + metastore: MetastoreServiceClient, +) -> Result { + info!(index_id = %target_index_id, "update-index"); + + let index_metadata_request = IndexMetadataRequest::for_index_id(target_index_id.to_string()); + let current_index_metadata = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; + let index_uid = current_index_metadata.index_uid.clone(); + let current_index_config = current_index_metadata.into_index_config(); + + let new_index_config = + load_index_config_update(config_format, &index_config_bytes, ¤t_index_config) + .map_err(IndexServiceError::InvalidConfig)?; + + let update_request = UpdateIndexRequest::try_from_updates( + index_uid, + &new_index_config.search_settings, + &new_index_config.retention_policy_opt, + &new_index_config.indexing_settings, + &new_index_config.doc_mapping, + )?; + let update_resp = metastore.update_index(update_request).await?; + Ok(update_resp.deserialize_index_metadata()?) +} + +pub fn clear_index_handler( + index_service: IndexService, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "clear") + .and(warp::put()) + .and(with_arg(index_service)) + .then(clear_index) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + put, + tag = "Indexes", + path = "/indexes/{index_id}/clear", + responses( + (status = 200, description = "Successfully cleared index.") + ), + params( + ("index_id" = String, Path, description = "The index ID to clear."), + ) +)] +/// Removes all of the data (splits, queued document) associated with the index, but keeps the index +/// configuration. (See also, `delete-index`). +pub async fn clear_index( + index_id: IndexId, + mut index_service: IndexService, +) -> Result<(), IndexServiceError> { + info!(index_id = %index_id, "clear-index"); + index_service.clear_index(&index_id).await +} + +#[derive(Deserialize, utoipa::IntoParams, utoipa::ToSchema)] +#[into_params(parameter_in = Query)] +pub struct DeleteIndexQueryParam { + #[serde(default)] + dry_run: bool, +} + +pub fn delete_index_handler( + index_service: IndexService, +) -> impl Filter + Clone { + warp::path!("indexes" / String) + .and(warp::delete()) + .and(serde_qs::warp::query(serde_qs::Config::default())) + .and(with_arg(index_service)) + .then(delete_index) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + delete, + tag = "Indexes", + path = "/indexes/{index_id}", + responses( + // We return `VersionedIndexMetadata` as it's the serialized model view. + (status = 200, description = "Successfully deleted index.", body = [FileEntry]) + ), + params( + DeleteIndexQueryParam, + ("index_id" = String, Path, description = "The index ID to delete."), + ) +)] +/// Deletes index. +pub async fn delete_index( + index_id: IndexId, + delete_index_query_param: DeleteIndexQueryParam, + mut index_service: IndexService, +) -> Result, IndexServiceError> { + info!(index_id = %index_id, dry_run = delete_index_query_param.dry_run, "delete-index"); + index_service + .delete_index(&index_id, delete_index_query_param.dry_run) + .await +} diff --git a/quickwit/quickwit-serve/src/index_api/mod.rs b/quickwit/quickwit-serve/src/index_api/mod.rs index a11878ac264..1870d79b856 100644 --- a/quickwit/quickwit-serve/src/index_api/mod.rs +++ b/quickwit/quickwit-serve/src/index_api/mod.rs @@ -17,9 +17,11 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . +mod index_resource; mod rest_handler; +mod source_resource; +mod split_resource; -pub use self::rest_handler::{ - get_index_metadata_handler, index_management_handlers, IndexApi, ListSplitsQueryParams, - ListSplitsResponse, -}; +pub use self::index_resource::get_index_metadata_handler; +pub use self::rest_handler::{index_management_handlers, IndexApi}; +pub use self::split_resource::{ListSplitsQueryParams, ListSplitsResponse}; diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index 0b33e730182..25e7d52bb43 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -19,39 +19,36 @@ use std::sync::Arc; -use bytes::Bytes; -use quickwit_common::uri::Uri; -use quickwit_config::{ - load_index_config_update, load_source_config_from_user_config, validate_index_id_pattern, - ConfigFormat, FileSourceParams, NodeConfig, SourceConfig, SourceParams, CLI_SOURCE_ID, - INGEST_API_SOURCE_ID, -}; +use quickwit_config::NodeConfig; use quickwit_doc_mapper::{analyze_text, TokenizerConfig}; use quickwit_index_management::{IndexService, IndexServiceError}; -use quickwit_metastore::{ - IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsQuery, - ListSplitsRequestExt, MetastoreServiceStreamSplitsExt, Split, SplitInfo, SplitState, - UpdateIndexRequestExt, -}; -use quickwit_proto::ingest::Shard; -use quickwit_proto::metastore::{ - DeleteSourceRequest, EntityKind, IndexMetadataRequest, ListIndexesMetadataRequest, - ListShardsRequest, ListShardsSubrequest, ListSplitsRequest, MarkSplitsForDeletionRequest, - MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, - ResetSourceCheckpointRequest, ToggleSourceRequest, UpdateIndexRequest, -}; -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}; -use tracing::{info, warn}; +use serde::Deserialize; +use tracing::warn; use warp::{Filter, Rejection}; -use crate::format::{extract_config_format, extract_format_from_qs}; +use super::get_index_metadata_handler; +use super::index_resource::{ + __path_clear_index, __path_create_index, __path_delete_index, __path_list_indexes_metadata, + __path_update_index, clear_index_handler, create_index_handler, delete_index_handler, + describe_index_handler, list_indexes_metadata_handler, update_index_handler, IndexStats, + __path_describe_index, +}; +use super::source_resource::{ + __path_create_source, __path_delete_source, __path_reset_source_checkpoint, + __path_toggle_source, create_source_handler, delete_source_handler, get_source_handler, + get_source_shards_handler, reset_source_checkpoint_handler, toggle_source_handler, + ToggleSource, +}; +use super::split_resource::{ + __path_list_splits, __path_mark_splits_for_deletion, list_splits_handler, + mark_splits_for_deletion_handler, SplitsForDeletion, +}; +use crate::format::extract_format_from_qs; use crate::rest::recover_fn; use crate::rest_api_response::into_rest_api_response; -use crate::simple_list::{from_simple_list, to_simple_list}; -use crate::with_arg; +use crate::simple_list::from_simple_list; #[derive(utoipa::OpenApi)] #[openapi( @@ -73,7 +70,7 @@ use crate::with_arg; )] pub struct IndexApi; -fn log_failure( +pub fn log_failure( message: &'static str, ) -> impl Fn(Result) -> Result + Clone { move |result| { @@ -84,6 +81,11 @@ fn log_failure( } } +pub fn json_body( +) -> impl Filter + Clone { + warp::body::content_length_limit(1024 * 1024).and(warp::body::json()) +} + pub fn index_management_handlers( index_service: IndexService, node_config: Arc, @@ -117,869 +119,6 @@ pub fn index_management_handlers( .boxed() } -fn json_body( -) -> impl Filter + Clone { - warp::body::content_length_limit(1024 * 1024).and(warp::body::json()) -} - -pub fn get_index_metadata_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String) - .and(warp::get()) - .and(with_arg(metastore)) - .then(get_index_metadata) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -async fn get_index_metadata( - index_id: IndexId, - metastore: MetastoreServiceClient, -) -> MetastoreResult { - info!(index_id = %index_id, "get-index-metadata"); - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_metadata = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()?; - Ok(index_metadata) -} - -/// This struct represents the QueryString passed to -/// the rest API to filter indexes. -#[derive(Debug, Clone, Deserialize, Serialize, utoipa::IntoParams, utoipa::ToSchema, Default)] -#[into_params(parameter_in = Query)] -pub struct ListIndexesQueryParams { - #[serde(deserialize_with = "from_simple_list")] - #[serde(skip_serializing_if = "Option::is_none")] - #[serde(default)] - pub index_id_patterns: Option>, -} - -fn list_indexes_metadata_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes") - .and(warp::get()) - .and(serde_qs::warp::query(serde_qs::Config::default())) - .and(with_arg(metastore)) - .then(list_indexes_metadata) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -/// Describes an index with its main information and statistics. -#[derive(Serialize, Deserialize, utoipa::ToSchema)] -struct IndexStats { - #[schema(value_type = String)] - pub index_id: IndexId, - #[schema(value_type = String)] - pub index_uri: Uri, - pub num_published_splits: usize, - pub size_published_splits: u64, - pub num_published_docs: u64, - pub size_published_docs_uncompressed: u64, - pub timestamp_field_name: Option, - pub min_timestamp: Option, - pub max_timestamp: Option, -} - -#[utoipa::path( - get, - tag = "Indexes", - path = "/indexes/{index_id}/describe", - responses( - (status = 200, description = "Successfully fetched stats about Index.", body = IndexStats) - ), - params( - ("index_id" = String, Path, description = "The index ID to describe."), - ) -)] - -/// Describes an index. -async fn describe_index( - index_id: IndexId, - metastore: MetastoreServiceClient, -) -> MetastoreResult { - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_metadata = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()?; - let query = ListSplitsQuery::for_index(index_metadata.index_uid.clone()); - let list_splits_request = ListSplitsRequest::try_from_list_splits_query(&query)?; - let splits = metastore - .list_splits(list_splits_request) - .await? - .collect_splits() - .await?; - let published_splits: Vec = splits - .into_iter() - .filter(|split| split.split_state == SplitState::Published) - .collect(); - let mut total_num_docs = 0; - let mut total_num_bytes = 0; - let mut total_uncompressed_num_bytes = 0; - let mut min_timestamp: Option = None; - let mut max_timestamp: Option = None; - - for split in &published_splits { - total_num_docs += split.split_metadata.num_docs as u64; - total_num_bytes += split.split_metadata.footer_offsets.end; - total_uncompressed_num_bytes += split.split_metadata.uncompressed_docs_size_in_bytes; - - if let Some(time_range) = &split.split_metadata.time_range { - min_timestamp = min_timestamp - .min(Some(*time_range.start())) - .or(Some(*time_range.start())); - max_timestamp = max_timestamp - .max(Some(*time_range.end())) - .or(Some(*time_range.end())); - } - } - - let index_config = index_metadata.into_index_config(); - let index_stats = IndexStats { - index_id, - index_uri: index_config.index_uri.clone(), - num_published_splits: published_splits.len(), - size_published_splits: total_num_bytes, - num_published_docs: total_num_docs, - size_published_docs_uncompressed: total_uncompressed_num_bytes, - timestamp_field_name: index_config.doc_mapping.timestamp_field, - min_timestamp, - max_timestamp, - }; - - Ok(index_stats) -} - -fn describe_index_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "describe") - .and(warp::get()) - .and(with_arg(metastore)) - .then(describe_index) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -/// This struct represents the QueryString passed to -/// the rest API to filter splits. -#[derive(Debug, Clone, Deserialize, Serialize, utoipa::IntoParams, utoipa::ToSchema, Default)] -#[into_params(parameter_in = Query)] -pub struct ListSplitsQueryParams { - /// If set, define the number of splits to skip - #[serde(skip_serializing_if = "Option::is_none")] - #[serde(default)] - pub offset: Option, - /// If set, restrict maximum number of splits to retrieve - #[serde(skip_serializing_if = "Option::is_none")] - #[serde(default)] - pub limit: Option, - /// A specific split state(s) to filter by. - #[serde(deserialize_with = "from_simple_list")] - #[serde(serialize_with = "to_simple_list")] - #[serde(skip_serializing_if = "Option::is_none")] - #[serde(default)] - pub split_states: Option>, - /// If set, restrict splits to documents with a `timestamp >= start_timestamp`. - /// This timestamp is in seconds. - #[serde(skip_serializing_if = "Option::is_none")] - #[serde(default)] - pub start_timestamp: Option, - /// If set, restrict splits to documents with a `timestamp < end_timestamp`. - /// This timestamp is in seconds. - #[serde(skip_serializing_if = "Option::is_none")] - #[serde(default)] - pub end_timestamp: Option, - /// If set, restrict splits whose creation dates are before this date. - #[serde(skip_serializing_if = "Option::is_none")] - #[serde(default)] - pub end_create_timestamp: Option, -} - -#[derive(Serialize, Deserialize, Debug, utoipa::ToSchema)] -pub struct ListSplitsResponse { - #[serde(default)] - pub offset: usize, - #[serde(default)] - pub size: usize, - #[serde(default)] - pub splits: Vec, -} - -#[utoipa::path( - get, - tag = "Indexes", - path = "/indexes/{index_id}/splits", - responses( - (status = 200, description = "Successfully fetched splits.", body = ListSplitsResponse) - ), - params( - ListSplitsQueryParams, - ("index_id" = String, Path, description = "The index ID to retrieve splits for."), - ) -)] - -/// Get splits. -async fn list_splits( - index_id: IndexId, - list_split_query: ListSplitsQueryParams, - metastore: MetastoreServiceClient, -) -> MetastoreResult { - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .index_uid; - info!(index_id = %index_id, list_split_query = ?list_split_query, "get-splits"); - let mut query = ListSplitsQuery::for_index(index_uid); - let mut offset = 0; - if let Some(offset_value) = list_split_query.offset { - query = query.with_offset(offset_value); - offset = offset_value; - } - if let Some(limit) = list_split_query.limit { - query = query.with_limit(limit); - } - if let Some(split_states) = list_split_query.split_states { - query = query.with_split_states(split_states); - } - if let Some(start_timestamp) = list_split_query.start_timestamp { - query = query.with_time_range_start_gte(start_timestamp); - } - if let Some(end_timestamp) = list_split_query.end_timestamp { - query = query.with_time_range_end_lt(end_timestamp); - } - if let Some(end_created_timestamp) = list_split_query.end_create_timestamp { - query = query.with_create_timestamp_lt(end_created_timestamp); - } - let list_splits_request = ListSplitsRequest::try_from_list_splits_query(&query)?; - let splits = metastore - .list_splits(list_splits_request) - .await? - .collect_splits() - .await?; - Ok(ListSplitsResponse { - offset, - size: splits.len(), - splits, - }) -} - -fn list_splits_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "splits") - .and(warp::get()) - .and(serde_qs::warp::query(serde_qs::Config::default())) - .and(with_arg(metastore)) - .then(list_splits) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[derive(Deserialize, utoipa::ToSchema)] -#[serde(deny_unknown_fields)] -struct SplitsForDeletion { - pub split_ids: Vec, -} - -#[utoipa::path( - put, - tag = "Splits", - path = "/indexes/{index_id}/splits/mark-for-deletion", - request_body = SplitsForDeletion, - responses( - (status = 200, description = "Successfully marked splits for deletion.") - ), - params( - ("index_id" = String, Path, description = "The index ID to mark splits for deletion for."), - ) -)] -/// Marks splits for deletion. -async fn mark_splits_for_deletion( - index_id: IndexId, - splits_for_deletion: SplitsForDeletion, - metastore: MetastoreServiceClient, -) -> MetastoreResult<()> { - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .index_uid; - info!(index_id = %index_id, splits_ids = ?splits_for_deletion.split_ids, "mark-splits-for-deletion"); - let split_ids: Vec = splits_for_deletion - .split_ids - .iter() - .map(|split_id| split_id.to_string()) - .collect(); - let mark_splits_for_deletion_request = - MarkSplitsForDeletionRequest::new(index_uid, split_ids.clone()); - metastore - .mark_splits_for_deletion(mark_splits_for_deletion_request) - .await?; - Ok(()) -} - -fn mark_splits_for_deletion_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "splits" / "mark-for-deletion") - .and(warp::put()) - .and(json_body()) - .and(with_arg(metastore)) - .then(mark_splits_for_deletion) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - get, - tag = "Indexes", - path = "/indexes", - responses( - // We return `VersionedIndexMetadata` as it's the serialized model view. - (status = 200, description = "Successfully fetched all indexes.", body = [VersionedIndexMetadata]) - ), - params( - ListIndexesQueryParams, - ("index_id_patterns" = String, Path, description = "The index ID pattern to retrieve indexes for."), - ) -)] -/// Gets indexes metadata. -async fn list_indexes_metadata( - list_indexes_params: ListIndexesQueryParams, - metastore: MetastoreServiceClient, -) -> MetastoreResult> { - let list_indexes_metata_request = - if let Some(index_id_patterns) = list_indexes_params.index_id_patterns { - for index_id_pattern in &index_id_patterns { - validate_index_id_pattern(index_id_pattern, true).map_err(|error| { - MetastoreError::InvalidArgument { - message: error.to_string(), - } - })?; - } - ListIndexesMetadataRequest { index_id_patterns } - } else { - ListIndexesMetadataRequest::all() - }; - metastore - .list_indexes_metadata(list_indexes_metata_request) - .await? - .deserialize_indexes_metadata() - .await -} - -#[derive(Deserialize, utoipa::IntoParams, utoipa::ToSchema)] -#[into_params(parameter_in = Query)] -struct CreateIndexQueryParams { - #[serde(default)] - overwrite: bool, -} - -fn create_index_handler( - index_service: IndexService, - node_config: Arc, -) -> impl Filter + Clone { - warp::path!("indexes") - .and(warp::post()) - .and(serde_qs::warp::query(serde_qs::Config::default())) - .and(extract_config_format()) - .and(warp::body::content_length_limit(1024 * 1024)) - .and(warp::filters::body::bytes()) - .and(with_arg(index_service)) - .and(with_arg(node_config)) - .then(create_index) - .map(log_failure("failed to create index")) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - post, - tag = "Indexes", - path = "/indexes", - request_body = VersionedIndexConfig, - responses( - // We return `VersionedIndexMetadata` as it's the serialized model view. - (status = 200, description = "Successfully created index.", body = VersionedIndexMetadata) - ), - params( - CreateIndexQueryParams, - ) -)] -/// Creates index. -async fn create_index( - create_index_query_params: CreateIndexQueryParams, - config_format: ConfigFormat, - index_config_bytes: Bytes, - mut index_service: IndexService, - node_config: Arc, -) -> Result { - let index_config = quickwit_config::load_index_config_from_user_config( - config_format, - &index_config_bytes, - &node_config.default_index_root_uri, - ) - .map_err(IndexServiceError::InvalidConfig)?; - info!(index_id = %index_config.index_id, overwrite = create_index_query_params.overwrite, "create-index"); - index_service - .create_index(index_config, create_index_query_params.overwrite) - .await -} - -fn update_index_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String) - .and(warp::put()) - .and(extract_config_format()) - .and(warp::body::content_length_limit(1024 * 1024)) - .and(warp::filters::body::bytes()) - .and(with_arg(metastore)) - .then(update_index) - .map(log_failure("failed to update index")) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - put, - tag = "Indexes", - path = "/indexes/{index_id}", - request_body = VersionedIndexConfig, - responses( - (status = 200, description = "Successfully updated the index configuration.", body = VersionedIndexMetadata) - ), - params( - ("index_id" = String, Path, description = "The index ID to update."), - ) -)] -/// Updates an existing index. -/// -/// This endpoint follows PUT semantics, which means that all the fields of the -/// current configuration are replaced by the values specified in this request -/// or the associated defaults. In particular, if the field is optional (e.g. -/// `retention_policy`), omitting it will delete the associated configuration. -/// If the new configuration file contains updates that cannot be applied, the -/// request fails, and none of the updates are applied. -async fn update_index( - target_index_id: IndexId, - config_format: ConfigFormat, - index_config_bytes: Bytes, - metastore: MetastoreServiceClient, -) -> Result { - info!(index_id = %target_index_id, "update-index"); - - let index_metadata_request = IndexMetadataRequest::for_index_id(target_index_id.to_string()); - let current_index_metadata = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()?; - let index_uid = current_index_metadata.index_uid.clone(); - let current_index_config = current_index_metadata.into_index_config(); - - let new_index_config = - load_index_config_update(config_format, &index_config_bytes, ¤t_index_config) - .map_err(IndexServiceError::InvalidConfig)?; - - let update_request = UpdateIndexRequest::try_from_updates( - index_uid, - &new_index_config.search_settings, - &new_index_config.retention_policy_opt, - &new_index_config.indexing_settings, - &new_index_config.doc_mapping, - )?; - let update_resp = metastore.update_index(update_request).await?; - Ok(update_resp.deserialize_index_metadata()?) -} - -fn clear_index_handler( - index_service: IndexService, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "clear") - .and(warp::put()) - .and(with_arg(index_service)) - .then(clear_index) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - put, - tag = "Indexes", - path = "/indexes/{index_id}/clear", - responses( - (status = 200, description = "Successfully cleared index.") - ), - params( - ("index_id" = String, Path, description = "The index ID to clear."), - ) -)] -/// 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: IndexId, - mut index_service: IndexService, -) -> Result<(), IndexServiceError> { - info!(index_id = %index_id, "clear-index"); - index_service.clear_index(&index_id).await -} - -#[derive(Deserialize, utoipa::IntoParams, utoipa::ToSchema)] -#[into_params(parameter_in = Query)] -struct DeleteIndexQueryParam { - #[serde(default)] - dry_run: bool, -} - -fn delete_index_handler( - index_service: IndexService, -) -> impl Filter + Clone { - warp::path!("indexes" / String) - .and(warp::delete()) - .and(serde_qs::warp::query(serde_qs::Config::default())) - .and(with_arg(index_service)) - .then(delete_index) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - delete, - tag = "Indexes", - path = "/indexes/{index_id}", - responses( - // We return `VersionedIndexMetadata` as it's the serialized model view. - (status = 200, description = "Successfully deleted index.", body = [FileEntry]) - ), - params( - DeleteIndexQueryParam, - ("index_id" = String, Path, description = "The index ID to delete."), - ) -)] -/// Deletes index. -async fn delete_index( - index_id: IndexId, - delete_index_query_param: DeleteIndexQueryParam, - mut index_service: IndexService, -) -> Result, IndexServiceError> { - info!(index_id = %index_id, dry_run = delete_index_query_param.dry_run, "delete-index"); - index_service - .delete_index(&index_id, delete_index_query_param.dry_run) - .await -} - -fn create_source_handler( - index_service: IndexService, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "sources") - .and(warp::post()) - .and(extract_config_format()) - .and(warp::body::content_length_limit(1024 * 1024)) - .and(warp::filters::body::bytes()) - .and(with_arg(index_service)) - .then(create_source) - .map(log_failure("failed to create source")) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - post, - tag = "Sources", - path = "/indexes/{index_id}/sources", - request_body = VersionedSourceConfig, - responses( - // We return `VersionedSourceConfig` as it's the serialized model view. - (status = 200, description = "Successfully created source.", body = VersionedSourceConfig) - ), - params( - ("index_id" = String, Path, description = "The index ID to create a source for."), - ) -)] -/// Creates Source. -async fn create_source( - index_id: IndexId, - config_format: ConfigFormat, - source_config_bytes: Bytes, - mut index_service: IndexService, -) -> Result { - let source_config: SourceConfig = - load_source_config_from_user_config(config_format, &source_config_bytes) - .map_err(IndexServiceError::InvalidConfig)?; - // Note: This check is performed here instead of the source config serde - // because many tests use the file source, and can't store that config in - // the metastore without going through the validation. - if let SourceParams::File(FileSourceParams::Filepath(_)) = &source_config.source_params { - return Err(IndexServiceError::InvalidConfig(anyhow::anyhow!( - "path based file sources are limited to a local usage, please use the CLI command \ - `quickwit tool local-ingest` to ingest data from a specific file or setup a \ - notification based file source" - ))); - } - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = index_service - .metastore() - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .index_uid; - info!(index_id = %index_id, source_id = %source_config.source_id, "create-source"); - index_service.add_source(index_uid, source_config).await -} - -fn get_source_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "sources" / String) - .and(warp::get()) - .and(with_arg(metastore)) - .then(get_source) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -async fn get_source( - index_id: IndexId, - source_id: SourceId, - metastore: MetastoreServiceClient, -) -> MetastoreResult { - info!(index_id = %index_id, source_id = %source_id, "get-source"); - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let source_config = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .sources - .remove(&source_id) - .ok_or({ - MetastoreError::NotFound(EntityKind::Source { - index_id, - source_id, - }) - })?; - Ok(source_config) -} - -fn reset_source_checkpoint_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "sources" / String / "reset-checkpoint") - .and(warp::put()) - .and(with_arg(metastore)) - .then(reset_source_checkpoint) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - put, - tag = "Sources", - path = "/indexes/{index_id}/sources/{source_id}/reset-checkpoint", - responses( - (status = 200, description = "Successfully reset source checkpoint.") - ), - params( - ("index_id" = String, Path, description = "The index ID of the source."), - ("source_id" = String, Path, description = "The source ID whose checkpoint is reset."), - ) -)] -/// Resets source checkpoint. -async fn reset_source_checkpoint( - index_id: IndexId, - source_id: SourceId, - metastore: MetastoreServiceClient, -) -> MetastoreResult<()> { - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .index_uid; - info!(index_id = %index_id, source_id = %source_id, "reset-checkpoint"); - let reset_source_checkpoint_request = ResetSourceCheckpointRequest { - index_uid: Some(index_uid), - source_id: source_id.clone(), - }; - metastore - .reset_source_checkpoint(reset_source_checkpoint_request) - .await?; - Ok(()) -} - -fn toggle_source_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "sources" / String / "toggle") - .and(warp::put()) - .and(json_body()) - .and(with_arg(metastore)) - .then(toggle_source) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[derive(Deserialize, utoipa::ToSchema)] -#[serde(deny_unknown_fields)] -struct ToggleSource { - enable: bool, -} - -#[utoipa::path( - put, - tag = "Sources", - path = "/indexes/{index_id}/sources/{source_id}/toggle", - request_body = ToggleSource, - responses( - (status = 200, description = "Successfully toggled source.") - ), - params( - ("index_id" = String, Path, description = "The index ID of the source."), - ("source_id" = String, Path, description = "The source ID to toggle."), - ) -)] -/// Toggles source. -async fn toggle_source( - index_id: IndexId, - source_id: SourceId, - toggle_source: ToggleSource, - metastore: MetastoreServiceClient, -) -> Result<(), IndexServiceError> { - info!(index_id = %index_id, source_id = %source_id, enable = toggle_source.enable, "toggle-source"); - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .index_uid; - if [CLI_SOURCE_ID, INGEST_API_SOURCE_ID].contains(&source_id.as_str()) { - return Err(IndexServiceError::OperationNotAllowed(format!( - "source `{source_id}` is managed by Quickwit, you cannot enable or disable a source \ - managed by Quickwit" - ))); - } - let toggle_source_request = ToggleSourceRequest { - index_uid: Some(index_uid), - source_id: source_id.clone(), - enable: toggle_source.enable, - }; - metastore.toggle_source(toggle_source_request).await?; - Ok(()) -} - -fn delete_source_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "sources" / String) - .and(warp::delete()) - .and(with_arg(metastore)) - .then(delete_source) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -#[utoipa::path( - delete, - tag = "Sources", - path = "/indexes/{index_id}/sources/{source_id}", - responses( - (status = 200, description = "Successfully deleted source.") - ), - params( - ("index_id" = String, Path, description = "The index ID to remove the source from."), - ("source_id" = String, Path, description = "The source ID to remove from the index."), - ) -)] -/// Deletes source. -async fn delete_source( - index_id: IndexId, - source_id: SourceId, - metastore: MetastoreServiceClient, -) -> Result<(), IndexServiceError> { - info!(index_id = %index_id, source_id = %source_id, "delete-source"); - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .index_uid; - if [INGEST_API_SOURCE_ID, CLI_SOURCE_ID].contains(&source_id.as_str()) { - return Err(IndexServiceError::OperationNotAllowed(format!( - "source `{source_id}` is managed by Quickwit, you cannot delete a source managed by \ - Quickwit" - ))); - } - let delete_source_request = DeleteSourceRequest { - index_uid: Some(index_uid), - source_id: source_id.clone(), - }; - metastore.delete_source(delete_source_request).await?; - Ok(()) -} - -fn get_source_shards_handler( - metastore: MetastoreServiceClient, -) -> impl Filter + Clone { - warp::path!("indexes" / String / "sources" / String / "shards") - .and(warp::get()) - .and(with_arg(metastore)) - .then(get_source_shards) - .and(extract_format_from_qs()) - .map(into_rest_api_response) - .boxed() -} - -async fn get_source_shards( - index_id: IndexId, - source_id: SourceId, - metastore: MetastoreServiceClient, -) -> MetastoreResult> { - info!(index_id = %index_id, source_id = %source_id, "get-source-shards"); - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = metastore - .index_metadata(index_metadata_request) - .await? - .deserialize_index_metadata()? - .index_uid; - let response = metastore - .list_shards(ListShardsRequest { - subrequests: vec![ListShardsSubrequest { - index_uid: Some(index_uid), - source_id: source_id.to_string(), - ..Default::default() - }], - }) - .await?; - let shards = response - .subresponses - .into_iter() - .flat_map(|resp| resp.shards) - .collect(); - Ok(shards) -} - #[derive(Debug, Deserialize, utoipa::IntoParams, utoipa::ToSchema)] struct AnalyzeRequest { /// The tokenizer to use. @@ -1074,13 +213,22 @@ mod tests { use assert_json_diff::assert_json_include; use quickwit_common::uri::Uri; use quickwit_common::ServiceStream; - use quickwit_config::{SourceParams, VecSourceParams}; + use quickwit_config::{ + NodeConfig, SourceParams, VecSourceParams, CLI_SOURCE_ID, INGEST_API_SOURCE_ID, + }; use quickwit_indexing::{mock_split, MockSplitBuilder}; - use quickwit_metastore::{metastore_for_test, IndexMetadata, ListSplitsResponseExt}; + use quickwit_metastore::{ + metastore_for_test, IndexMetadata, IndexMetadataResponseExt, + ListIndexesMetadataResponseExt, ListSplitsRequestExt, ListSplitsResponseExt, SplitState, + }; use quickwit_proto::metastore::{ - EmptyResponse, IndexMetadataResponse, ListIndexesMetadataResponse, ListSplitsResponse, - MetastoreServiceClient, MockMetastoreService, SourceType, + DeleteSourceRequest, EmptyResponse, EntityKind, IndexMetadataRequest, + IndexMetadataResponse, ListIndexesMetadataRequest, ListIndexesMetadataResponse, + ListSplitsRequest, ListSplitsResponse, MarkSplitsForDeletionRequest, MetastoreError, + MetastoreService, MetastoreServiceClient, MockMetastoreService, + ResetSourceCheckpointRequest, SourceType, ToggleSourceRequest, }; + use quickwit_proto::types::IndexUid; use quickwit_storage::StorageResolver; use serde_json::Value as JsonValue; diff --git a/quickwit/quickwit-serve/src/index_api/source_resource.rs b/quickwit/quickwit-serve/src/index_api/source_resource.rs new file mode 100644 index 00000000000..2ece76047b3 --- /dev/null +++ b/quickwit/quickwit-serve/src/index_api/source_resource.rs @@ -0,0 +1,335 @@ +// Copyright (C) 2024 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use bytes::Bytes; +use quickwit_config::{ + load_source_config_from_user_config, ConfigFormat, FileSourceParams, SourceConfig, + SourceParams, CLI_SOURCE_ID, INGEST_API_SOURCE_ID, +}; +use quickwit_index_management::{IndexService, IndexServiceError}; +use quickwit_metastore::IndexMetadataResponseExt; +use quickwit_proto::ingest::Shard; +use quickwit_proto::metastore::{ + DeleteSourceRequest, EntityKind, IndexMetadataRequest, ListShardsRequest, ListShardsSubrequest, + MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, + ResetSourceCheckpointRequest, ToggleSourceRequest, +}; +use quickwit_proto::types::{IndexId, IndexUid, SourceId}; +use serde::Deserialize; +use tracing::info; +use warp::{Filter, Rejection}; + +use super::rest_handler::{json_body, log_failure}; +use crate::format::{extract_config_format, extract_format_from_qs}; +use crate::rest_api_response::into_rest_api_response; +use crate::with_arg; + +pub fn create_source_handler( + index_service: IndexService, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "sources") + .and(warp::post()) + .and(extract_config_format()) + .and(warp::body::content_length_limit(1024 * 1024)) + .and(warp::filters::body::bytes()) + .and(with_arg(index_service)) + .then(create_source) + .map(log_failure("failed to create source")) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + post, + tag = "Sources", + path = "/indexes/{index_id}/sources", + request_body = VersionedSourceConfig, + responses( + // We return `VersionedSourceConfig` as it's the serialized model view. + (status = 200, description = "Successfully created source.", body = VersionedSourceConfig) + ), + params( + ("index_id" = String, Path, description = "The index ID to create a source for."), + ) +)] +/// Creates Source. +pub async fn create_source( + index_id: IndexId, + config_format: ConfigFormat, + source_config_bytes: Bytes, + mut index_service: IndexService, +) -> Result { + let source_config: SourceConfig = + load_source_config_from_user_config(config_format, &source_config_bytes) + .map_err(IndexServiceError::InvalidConfig)?; + // Note: This check is performed here instead of the source config serde + // because many tests use the file source, and can't store that config in + // the metastore without going through the validation. + if let SourceParams::File(FileSourceParams::Filepath(_)) = &source_config.source_params { + return Err(IndexServiceError::InvalidConfig(anyhow::anyhow!( + "path based file sources are limited to a local usage, please use the CLI command \ + `quickwit tool local-ingest` to ingest data from a specific file or setup a \ + notification based file source" + ))); + } + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = index_service + .metastore() + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + info!(index_id = %index_id, source_id = %source_config.source_id, "create-source"); + index_service.add_source(index_uid, source_config).await +} + +pub fn get_source_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "sources" / String) + .and(warp::get()) + .and(with_arg(metastore)) + .then(get_source) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +pub async fn get_source( + index_id: IndexId, + source_id: SourceId, + metastore: MetastoreServiceClient, +) -> MetastoreResult { + info!(index_id = %index_id, source_id = %source_id, "get-source"); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let source_config = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .sources + .remove(&source_id) + .ok_or({ + MetastoreError::NotFound(EntityKind::Source { + index_id, + source_id, + }) + })?; + Ok(source_config) +} + +pub fn reset_source_checkpoint_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "sources" / String / "reset-checkpoint") + .and(warp::put()) + .and(with_arg(metastore)) + .then(reset_source_checkpoint) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + put, + tag = "Sources", + path = "/indexes/{index_id}/sources/{source_id}/reset-checkpoint", + responses( + (status = 200, description = "Successfully reset source checkpoint.") + ), + params( + ("index_id" = String, Path, description = "The index ID of the source."), + ("source_id" = String, Path, description = "The source ID whose checkpoint is reset."), + ) +)] +/// Resets source checkpoint. +pub async fn reset_source_checkpoint( + index_id: IndexId, + source_id: SourceId, + metastore: MetastoreServiceClient, +) -> MetastoreResult<()> { + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + info!(index_id = %index_id, source_id = %source_id, "reset-checkpoint"); + let reset_source_checkpoint_request = ResetSourceCheckpointRequest { + index_uid: Some(index_uid), + source_id: source_id.clone(), + }; + metastore + .reset_source_checkpoint(reset_source_checkpoint_request) + .await?; + Ok(()) +} + +pub fn toggle_source_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "sources" / String / "toggle") + .and(warp::put()) + .and(json_body()) + .and(with_arg(metastore)) + .then(toggle_source) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[derive(Deserialize, utoipa::ToSchema)] +#[serde(deny_unknown_fields)] +pub struct ToggleSource { + enable: bool, +} + +#[utoipa::path( + put, + tag = "Sources", + path = "/indexes/{index_id}/sources/{source_id}/toggle", + request_body = ToggleSource, + responses( + (status = 200, description = "Successfully toggled source.") + ), + params( + ("index_id" = String, Path, description = "The index ID of the source."), + ("source_id" = String, Path, description = "The source ID to toggle."), + ) +)] +/// Toggles source. +pub async fn toggle_source( + index_id: IndexId, + source_id: SourceId, + toggle_source: ToggleSource, + metastore: MetastoreServiceClient, +) -> Result<(), IndexServiceError> { + info!(index_id = %index_id, source_id = %source_id, enable = toggle_source.enable, "toggle-source"); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + if [CLI_SOURCE_ID, INGEST_API_SOURCE_ID].contains(&source_id.as_str()) { + return Err(IndexServiceError::OperationNotAllowed(format!( + "source `{source_id}` is managed by Quickwit, you cannot enable or disable a source \ + managed by Quickwit" + ))); + } + let toggle_source_request = ToggleSourceRequest { + index_uid: Some(index_uid), + source_id: source_id.clone(), + enable: toggle_source.enable, + }; + metastore.toggle_source(toggle_source_request).await?; + Ok(()) +} + +pub fn delete_source_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "sources" / String) + .and(warp::delete()) + .and(with_arg(metastore)) + .then(delete_source) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + delete, + tag = "Sources", + path = "/indexes/{index_id}/sources/{source_id}", + responses( + (status = 200, description = "Successfully deleted source.") + ), + params( + ("index_id" = String, Path, description = "The index ID to remove the source from."), + ("source_id" = String, Path, description = "The source ID to remove from the index."), + ) +)] +/// Deletes source. +pub async fn delete_source( + index_id: IndexId, + source_id: SourceId, + metastore: MetastoreServiceClient, +) -> Result<(), IndexServiceError> { + info!(index_id = %index_id, source_id = %source_id, "delete-source"); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + if [INGEST_API_SOURCE_ID, CLI_SOURCE_ID].contains(&source_id.as_str()) { + return Err(IndexServiceError::OperationNotAllowed(format!( + "source `{source_id}` is managed by Quickwit, you cannot delete a source managed by \ + Quickwit" + ))); + } + let delete_source_request = DeleteSourceRequest { + index_uid: Some(index_uid), + source_id: source_id.clone(), + }; + metastore.delete_source(delete_source_request).await?; + Ok(()) +} + +pub fn get_source_shards_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "sources" / String / "shards") + .and(warp::get()) + .and(with_arg(metastore)) + .then(get_source_shards) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +pub async fn get_source_shards( + index_id: IndexId, + source_id: SourceId, + metastore: MetastoreServiceClient, +) -> MetastoreResult> { + info!(index_id = %index_id, source_id = %source_id, "get-source-shards"); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + let response = metastore + .list_shards(ListShardsRequest { + subrequests: vec![ListShardsSubrequest { + index_uid: Some(index_uid), + source_id: source_id.to_string(), + ..Default::default() + }], + }) + .await?; + let shards = response + .subresponses + .into_iter() + .flat_map(|resp| resp.shards) + .collect(); + Ok(shards) +} diff --git a/quickwit/quickwit-serve/src/index_api/split_resource.rs b/quickwit/quickwit-serve/src/index_api/split_resource.rs new file mode 100644 index 00000000000..4eb3fbe5f33 --- /dev/null +++ b/quickwit/quickwit-serve/src/index_api/split_resource.rs @@ -0,0 +1,212 @@ +// Copyright (C) 2024 Quickwit, Inc. +// +// Quickwit is offered under the AGPL v3.0 and as commercial software. +// For commercial licensing, contact us at hello@quickwit.io. +// +// AGPL: +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU Affero General Public License as +// published by the Free Software Foundation, either version 3 of the +// License, or (at your option) any later version. +// +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Affero General Public License for more details. +// +// You should have received a copy of the GNU Affero General Public License +// along with this program. If not, see . + +use quickwit_metastore::{ + IndexMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, + MetastoreServiceStreamSplitsExt, Split, SplitState, +}; +use quickwit_proto::metastore::{ + IndexMetadataRequest, ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreResult, + MetastoreService, MetastoreServiceClient, +}; +use quickwit_proto::types::{IndexId, IndexUid}; +use serde::{Deserialize, Serialize}; +use tracing::info; +use warp::{Filter, Rejection}; + +use super::rest_handler::json_body; +use crate::format::extract_format_from_qs; +use crate::rest_api_response::into_rest_api_response; +use crate::simple_list::{from_simple_list, to_simple_list}; +use crate::with_arg; + +/// This struct represents the QueryString passed to +/// the rest API to filter splits. +#[derive(Debug, Clone, Deserialize, Serialize, utoipa::IntoParams, utoipa::ToSchema, Default)] +#[into_params(parameter_in = Query)] +pub struct ListSplitsQueryParams { + /// If set, define the number of splits to skip + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub offset: Option, + /// If set, restrict maximum number of splits to retrieve + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub limit: Option, + /// A specific split state(s) to filter by. + #[serde(deserialize_with = "from_simple_list")] + #[serde(serialize_with = "to_simple_list")] + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub split_states: Option>, + /// If set, restrict splits to documents with a `timestamp >= start_timestamp`. + /// This timestamp is in seconds. + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub start_timestamp: Option, + /// If set, restrict splits to documents with a `timestamp < end_timestamp`. + /// This timestamp is in seconds. + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub end_timestamp: Option, + /// If set, restrict splits whose creation dates are before this date. + #[serde(skip_serializing_if = "Option::is_none")] + #[serde(default)] + pub end_create_timestamp: Option, +} + +#[derive(Serialize, Deserialize, Debug, utoipa::ToSchema)] +pub struct ListSplitsResponse { + #[serde(default)] + pub offset: usize, + #[serde(default)] + pub size: usize, + #[serde(default)] + pub splits: Vec, +} + +#[utoipa::path( + get, + tag = "Indexes", + path = "/indexes/{index_id}/splits", + responses( + (status = 200, description = "Successfully fetched splits.", body = ListSplitsResponse) + ), + params( + ListSplitsQueryParams, + ("index_id" = String, Path, description = "The index ID to retrieve splits for."), + ) +)] + +/// Get splits. +pub async fn list_splits( + index_id: IndexId, + list_split_query: ListSplitsQueryParams, + metastore: MetastoreServiceClient, +) -> MetastoreResult { + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + info!(index_id = %index_id, list_split_query = ?list_split_query, "get-splits"); + let mut query = ListSplitsQuery::for_index(index_uid); + let mut offset = 0; + if let Some(offset_value) = list_split_query.offset { + query = query.with_offset(offset_value); + offset = offset_value; + } + if let Some(limit) = list_split_query.limit { + query = query.with_limit(limit); + } + if let Some(split_states) = list_split_query.split_states { + query = query.with_split_states(split_states); + } + if let Some(start_timestamp) = list_split_query.start_timestamp { + query = query.with_time_range_start_gte(start_timestamp); + } + if let Some(end_timestamp) = list_split_query.end_timestamp { + query = query.with_time_range_end_lt(end_timestamp); + } + if let Some(end_created_timestamp) = list_split_query.end_create_timestamp { + query = query.with_create_timestamp_lt(end_created_timestamp); + } + let list_splits_request = ListSplitsRequest::try_from_list_splits_query(&query)?; + let splits = metastore + .list_splits(list_splits_request) + .await? + .collect_splits() + .await?; + Ok(ListSplitsResponse { + offset, + size: splits.len(), + splits, + }) +} + +pub fn list_splits_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "splits") + .and(warp::get()) + .and(serde_qs::warp::query(serde_qs::Config::default())) + .and(with_arg(metastore)) + .then(list_splits) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[derive(Deserialize, utoipa::ToSchema)] +#[serde(deny_unknown_fields)] +pub struct SplitsForDeletion { + pub split_ids: Vec, +} + +#[utoipa::path( + put, + tag = "Splits", + path = "/indexes/{index_id}/splits/mark-for-deletion", + request_body = SplitsForDeletion, + responses( + (status = 200, description = "Successfully marked splits for deletion.") + ), + params( + ("index_id" = String, Path, description = "The index ID to mark splits for deletion for."), + ) +)] +/// Marks splits for deletion. +pub async fn mark_splits_for_deletion( + index_id: IndexId, + splits_for_deletion: SplitsForDeletion, + metastore: MetastoreServiceClient, +) -> MetastoreResult<()> { + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let index_uid: IndexUid = metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .index_uid; + info!(index_id = %index_id, splits_ids = ?splits_for_deletion.split_ids, "mark-splits-for-deletion"); + let split_ids: Vec = splits_for_deletion + .split_ids + .iter() + .map(|split_id| split_id.to_string()) + .collect(); + let mark_splits_for_deletion_request = + MarkSplitsForDeletionRequest::new(index_uid, split_ids.clone()); + metastore + .mark_splits_for_deletion(mark_splits_for_deletion_request) + .await?; + Ok(()) +} + +pub fn mark_splits_for_deletion_handler( + metastore: MetastoreServiceClient, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "splits" / "mark-for-deletion") + .and(warp::put()) + .and(json_body()) + .and(with_arg(metastore)) + .then(mark_splits_for_deletion) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} From d0754c8eca01cf59e7141c5209f4fecc3ccf67ed Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:56 +0100 Subject: [PATCH 03/11] Add the source to the indexing pipeline's param fingerprint Currently the pipeline fingerpring only contains index settings. The source setting changes should also trigger a pipeline restart. Note that this change will change all the existing pipeline fingerprints. This means that once the control plane runs this for the first time, it will restart all indexing pipelines. --- .../quickwit-config/src/index_config/mod.rs | 11 +++- quickwit/quickwit-config/src/lib.rs | 12 +++++ .../quickwit-config/src/source_config/mod.rs | 52 +++++++++++++------ .../src/indexing_scheduler/mod.rs | 4 +- .../quickwit-control-plane/src/model/mod.rs | 3 +- .../src/actors/indexing_service.rs | 5 +- .../protos/quickwit/indexing.proto | 2 +- .../src/codegen/quickwit/quickwit.indexing.rs | 2 +- 8 files changed, 67 insertions(+), 24 deletions(-) diff --git a/quickwit/quickwit-config/src/index_config/mod.rs b/quickwit/quickwit-config/src/index_config/mod.rs index b2ee0839b8a..fcef34a9366 100644 --- a/quickwit/quickwit-config/src/index_config/mod.rs +++ b/quickwit/quickwit-config/src/index_config/mod.rs @@ -262,13 +262,22 @@ pub struct IndexConfig { impl IndexConfig { /// Return a fingerprint of parameters relevant for indexers - pub fn indexing_params_fingerprint(&self) -> u64 { + /// + /// This should remain private to this crate to avoid confusion with the + /// full indexing pipeline fingerprint that also includes the source's + /// fingerprint. + pub(crate) fn indexing_params_fingerprint(&self) -> u64 { let mut hasher = SipHasher::new(); self.doc_mapping.doc_mapping_uid.hash(&mut hasher); self.indexing_settings.hash(&mut hasher); hasher.finish() } + /// Compare IndexConfig config level fingerprints + pub fn equals_fingerprint(&self, other: &Self) -> bool { + self.indexing_params_fingerprint() == other.indexing_params_fingerprint() + } + #[cfg(any(test, feature = "testsuite"))] pub fn for_test(index_id: &str, index_uri: &str) -> Self { let index_uri = Uri::from_str(index_uri).unwrap(); diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index dd6f4612482..17f0dea7b29 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -19,6 +19,7 @@ #![deny(clippy::disallowed_methods)] +use std::hash::Hasher; use std::str::FromStr; use anyhow::{bail, ensure, Context}; @@ -55,6 +56,7 @@ pub use quickwit_doc_mapper::DocMapping; use serde::de::DeserializeOwned; use serde::Serialize; use serde_json::Value as JsonValue; +use siphasher::sip::SipHasher; use source_config::FileSourceParamsForSerde; pub use source_config::{ load_source_config_from_user_config, FileSourceMessageType, FileSourceNotification, @@ -286,6 +288,16 @@ pub trait TestableForRegression: Serialize + DeserializeOwned { fn assert_equality(&self, other: &Self); } +pub fn indexing_params_fingerprint( + index_config: &IndexConfig, + source_config: &SourceConfig, +) -> u64 { + let mut hasher = SipHasher::new(); + hasher.write_u64(index_config.indexing_params_fingerprint()); + hasher.write_u64(source_config.indexing_params_fingerprint()); + hasher.finish() +} + #[cfg(test)] mod tests { use super::validate_identifier; diff --git a/quickwit/quickwit-config/src/source_config/mod.rs b/quickwit/quickwit-config/src/source_config/mod.rs index de0790004d3..d8395ae5fe8 100644 --- a/quickwit/quickwit-config/src/source_config/mod.rs +++ b/quickwit/quickwit-config/src/source_config/mod.rs @@ -20,6 +20,7 @@ pub(crate) mod serialize; use std::borrow::Cow; +use std::hash::{Hash, Hasher}; use std::num::NonZeroUsize; use std::str::FromStr; @@ -35,6 +36,7 @@ use serde_json::Value as JsonValue; pub use serialize::load_source_config_from_user_config; // For backward compatibility. use serialize::VersionedSourceConfig; +use siphasher::sip::SipHasher; use crate::{disable_ingest_v1, enable_ingest_v2}; @@ -143,6 +145,20 @@ impl SourceConfig { } } + /// Return a fingerprint of parameters relevant for indexers + /// + /// This should remain private to this crate to avoid confusion with the + /// full indexing pipeline fingerprint that also includes the index config's + /// fingerprint. + pub(crate) fn indexing_params_fingerprint(&self) -> u64 { + let mut hasher = SipHasher::new(); + self.input_format.hash(&mut hasher); + self.num_pipelines.hash(&mut hasher); + self.source_params.hash(&mut hasher); + self.transform_config.hash(&mut hasher); + hasher.finish() + } + #[cfg(any(test, feature = "testsuite"))] pub fn for_test(source_id: &str, source_params: SourceParams) -> Self { Self { @@ -182,7 +198,9 @@ impl crate::TestableForRegression for SourceConfig { } } -#[derive(Clone, Copy, Debug, Default, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive( + Clone, Copy, Debug, Default, Eq, PartialEq, Serialize, Deserialize, Hash, utoipa::ToSchema, +)] #[serde(rename_all = "snake_case")] pub enum SourceInputFormat { #[default] @@ -214,7 +232,7 @@ impl FromStr for SourceInputFormat { } } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, Hash, utoipa::ToSchema)] #[serde(tag = "source_type", content = "params", rename_all = "snake_case")] pub enum SourceParams { #[schema(value_type = FileSourceParamsForSerde)] @@ -252,7 +270,7 @@ impl SourceParams { } } -#[derive(Clone, Copy, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Copy, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] pub enum FileSourceMessageType { /// See @@ -261,7 +279,7 @@ pub enum FileSourceMessageType { RawUri, } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] pub struct FileSourceSqs { pub queue_url: String, pub message_type: FileSourceMessageType, @@ -285,7 +303,7 @@ fn default_deduplication_cleanup_interval_secs() -> u32 { 60 } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(tag = "type", rename_all = "snake_case")] pub enum FileSourceNotification { Sqs(FileSourceSqs), @@ -300,7 +318,7 @@ pub(super) struct FileSourceParamsForSerde { filepath: Option, } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize)] #[serde( try_from = "FileSourceParamsForSerde", into = "FileSourceParamsForSerde" @@ -359,7 +377,7 @@ impl FileSourceParams { } } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct KafkaSourceParams { /// Name of the topic that the source consumes. @@ -379,7 +397,7 @@ pub struct KafkaSourceParams { pub enable_backfill_mode: bool, } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct PubSubSourceParams { /// Name of the subscription that the source consumes. @@ -401,14 +419,14 @@ pub struct PubSubSourceParams { pub max_messages_per_pull: Option, } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(rename_all = "lowercase")] pub enum RegionOrEndpoint { Region(String), Endpoint(String), } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(try_from = "KinesisSourceParamsInner")] pub struct KinesisSourceParams { pub stream_name: String, @@ -448,7 +466,7 @@ impl TryFrom for KinesisSourceParams { } } -#[derive(Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Default, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct VecSourceParams { #[schema(value_type = Vec)] @@ -458,11 +476,13 @@ pub struct VecSourceParams { pub partition: String, } -#[derive(Clone, Debug, Eq, PartialEq, Serialize, Deserialize, utoipa::ToSchema)] +#[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct VoidSourceParams; -#[derive(Clone, Debug, Eq, PartialEq, serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive( + Clone, Debug, Eq, PartialEq, Hash, serde::Serialize, serde::Deserialize, utoipa::ToSchema, +)] #[serde(deny_unknown_fields)] pub struct PulsarSourceParams { /// List of the topics that the source consumes. @@ -483,7 +503,9 @@ pub struct PulsarSourceParams { pub authentication: Option, } -#[derive(Clone, Debug, Eq, PartialEq, serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive( + Clone, Debug, Eq, PartialEq, Hash, serde::Serialize, serde::Deserialize, utoipa::ToSchema, +)] #[serde(rename_all = "lowercase")] pub enum PulsarSourceAuth { Token(String), @@ -515,7 +537,7 @@ fn default_consumer_name() -> String { "quickwit".to_string() } -#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, utoipa::ToSchema)] +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq, Eq, Hash, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct TransformConfig { /// [VRL] source code of the transform compiled to a VRL [`Program`](vrl::compiler::Program). diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs index 25159da52e3..a64f1948c12 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs @@ -30,7 +30,7 @@ use fnv::{FnvHashMap, FnvHashSet}; use itertools::Itertools; use once_cell::sync::OnceCell; use quickwit_common::pretty::PrettySample; -use quickwit_config::{FileSourceParams, SourceParams}; +use quickwit_config::{indexing_params_fingerprint, FileSourceParams, SourceParams}; use quickwit_proto::indexing::{ ApplyIndexingPlanRequest, CpuCapacity, IndexingService, IndexingTask, PIPELINE_FULL_CAPACITY, PIPELINE_THROUGHPUT, @@ -170,7 +170,7 @@ fn get_sources_to_schedule(model: &ControlPlaneModel) -> Vec { } let params_fingerprint = model .index_metadata(&source_uid.index_uid) - .map(|index_meta| index_meta.index_config.indexing_params_fingerprint()) + .map(|index_meta| indexing_params_fingerprint(&index_meta.index_config, source_config)) .unwrap_or_default(); match source_config.source_params { SourceParams::File(FileSourceParams::Filepath(_)) diff --git a/quickwit/quickwit-control-plane/src/model/mod.rs b/quickwit/quickwit-control-plane/src/model/mod.rs index 6312e51a5f9..7a574e97515 100644 --- a/quickwit/quickwit-control-plane/src/model/mod.rs +++ b/quickwit/quickwit-control-plane/src/model/mod.rs @@ -219,8 +219,7 @@ impl ControlPlaneModel { let Some(index_model) = self.index_table.get_mut(index_uid) else { bail!("index `{}` not found", index_uid.index_id); }; - let fp_changed = index_model.index_config.indexing_params_fingerprint() - != index_config.indexing_params_fingerprint(); + let fp_changed = !index_model.index_config.equals_fingerprint(&index_config); index_model.index_config = index_config; self.update_metrics(); Ok(fp_changed) diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 09530277194..7eecae37e95 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -36,7 +36,8 @@ use quickwit_common::io::Limiter; use quickwit_common::pubsub::EventBroker; use quickwit_common::{io, temp_dir}; use quickwit_config::{ - build_doc_mapper, IndexConfig, IndexerConfig, SourceConfig, INGEST_API_SOURCE_ID, + build_doc_mapper, indexing_params_fingerprint, IndexConfig, IndexerConfig, SourceConfig, + INGEST_API_SOURCE_ID, }; use quickwit_ingest::{ DropQueueRequest, GetPartitionId, IngestApiService, IngesterPool, ListQueuesRequest, @@ -315,7 +316,7 @@ impl IndexingService { let max_concurrent_split_uploads_merge = (self.max_concurrent_split_uploads - max_concurrent_split_uploads_index).max(1); - let params_fingerprint = index_config.indexing_params_fingerprint(); + let params_fingerprint = indexing_params_fingerprint(&index_config, &source_config); let pipeline_params = IndexingPipelineParams { pipeline_id: indexing_pipeline_id.clone(), metastore: self.metastore.clone(), diff --git a/quickwit/quickwit-proto/protos/quickwit/indexing.proto b/quickwit/quickwit-proto/protos/quickwit/indexing.proto index 294f301bc4b..aae849ec348 100644 --- a/quickwit/quickwit-proto/protos/quickwit/indexing.proto +++ b/quickwit/quickwit-proto/protos/quickwit/indexing.proto @@ -47,7 +47,7 @@ message IndexingTask { // The shards assigned to the indexer. repeated quickwit.ingest.ShardId shard_ids = 3; // Fingerprint of the pipeline parameters. Anything that should cause a pipeline restart (such - // as updating indexing settings or doc mapping) should influence this value. + // as updating indexing settings, the doc mapping or the source) should influence this value. uint64 params_fingerprint = 6; } diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs index ae0ef465968..34492422459 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs @@ -22,7 +22,7 @@ pub struct IndexingTask { #[prost(message, repeated, tag = "3")] pub shard_ids: ::prost::alloc::vec::Vec, /// Fingerprint of the pipeline parameters. Anything that should cause a pipeline restart (such - /// as updating indexing settings or doc mapping) should influence this value. + /// as updating indexing settings, the doc mapping or the source) should influence this value. #[prost(uint64, tag = "6")] pub params_fingerprint: u64, } From 99f324e09eb0bc3df4e9e309cbb738ad89ab6bcb Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:57 +0100 Subject: [PATCH 04/11] Validate fingerprints in indexing service --- .../src/actors/indexing_service.rs | 73 ++++++++++++------- 1 file changed, 46 insertions(+), 27 deletions(-) diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 7eecae37e95..b981c3d28ff 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -247,8 +247,15 @@ impl IndexingService { pipeline_uid, }; let index_config = index_metadata.into_index_config(); - self.spawn_pipeline_inner(ctx, pipeline_id.clone(), index_config, source_config, None) - .await?; + self.spawn_pipeline_inner( + ctx, + pipeline_id.clone(), + index_config, + source_config, + None, + None, + ) + .await?; Ok(pipeline_id) } @@ -259,6 +266,7 @@ impl IndexingService { index_config: IndexConfig, source_config: SourceConfig, immature_splits_opt: Option>, + expected_params_fingerprint: Option, ) -> Result<(), IndexingError> { if self .indexing_pipelines @@ -317,6 +325,15 @@ impl IndexingService { (self.max_concurrent_split_uploads - max_concurrent_split_uploads_index).max(1); let params_fingerprint = indexing_params_fingerprint(&index_config, &source_config); + if let Some(expected_params_fingerprint) = expected_params_fingerprint { + if params_fingerprint != expected_params_fingerprint { + warn!( + expected_fingerprint = expected_params_fingerprint, + actual_fingerprint = params_fingerprint, + "params fingerprint mismatch" + ); + } + } let pipeline_params = IndexingPipelineParams { pipeline_id: indexing_pipeline_id.clone(), metastore: self.metastore.clone(), @@ -637,20 +654,14 @@ impl IndexingService { /// Identifies the pipelines to spawn and shutdown by comparing the scheduled plan with the /// current running plan. fn compute_pipeline_diff(&self, tasks: &[IndexingTask]) -> IndexingPipelineDiff { - let mut pipelines_to_spawn: Vec = Vec::new(); + let mut pipelines_to_spawn: Vec = Vec::new(); let mut scheduled_pipeline_uids: HashSet = HashSet::with_capacity(tasks.len()); for task in tasks { let pipeline_uid = task.pipeline_uid(); if !self.indexing_pipelines.contains_key(&pipeline_uid) { - let pipeline_id = IndexingPipelineId { - node_id: self.node_id.clone(), - index_uid: task.index_uid().clone(), - source_id: task.source_id.clone(), - pipeline_uid, - }; - pipelines_to_spawn.push(pipeline_id); + pipelines_to_spawn.push(task.clone()); } scheduled_pipeline_uids.insert(pipeline_uid); } @@ -670,10 +681,19 @@ impl IndexingService { /// Spawns the pipelines with supplied ids and returns a list of failed pipelines. async fn spawn_pipelines( &mut self, - pipelines_to_spawn: &[IndexingPipelineId], + pipelines_to_spawn: &[IndexingTask], ctx: &ActorContext, ) -> Result, IndexingError> { - let indexes_metadata = self.indexes_metadata(ctx, pipelines_to_spawn).await?; + let pipelines_to_spawn_ids: Vec<_> = pipelines_to_spawn + .iter() + .map(|task| IndexingPipelineId { + node_id: self.node_id.clone(), + index_uid: task.index_uid().clone(), + pipeline_uid: task.pipeline_uid(), + source_id: task.source_id.clone(), + }) + .collect(); + let indexes_metadata = self.indexes_metadata(ctx, &pipelines_to_spawn_ids).await?; let per_index_uid_indexes_metadata: HashMap = indexes_metadata .into_iter() @@ -681,45 +701,44 @@ impl IndexingService { .collect(); let mut per_merge_pipeline_immature_splits: HashMap> = - self.fetch_immature_splits_for_new_merge_pipelines(pipelines_to_spawn, ctx) + self.fetch_immature_splits_for_new_merge_pipelines(&pipelines_to_spawn_ids, ctx) .await?; let mut spawn_pipeline_failures: Vec = Vec::new(); - for pipeline_to_spawn in pipelines_to_spawn { + for (task_to_spawn, id_to_spawn) in pipelines_to_spawn.iter().zip(pipelines_to_spawn_ids) { if let Some(index_metadata) = - per_index_uid_indexes_metadata.get(&pipeline_to_spawn.index_uid) + per_index_uid_indexes_metadata.get(task_to_spawn.index_uid()) { - if let Some(source_config) = - index_metadata.sources.get(&pipeline_to_spawn.source_id) - { - let merge_pipeline_id = pipeline_to_spawn.merge_pipeline_id(); + if let Some(source_config) = index_metadata.sources.get(&task_to_spawn.source_id) { + let merge_pipeline_id = id_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(), + id_to_spawn.clone(), index_metadata.index_config.clone(), source_config.clone(), immature_splits_opt, + Some(task_to_spawn.params_fingerprint), ) .await { - error!(pipeline_id=?pipeline_to_spawn, %error, "failed to spawn pipeline"); - spawn_pipeline_failures.push(pipeline_to_spawn.clone()); + error!(pipeline_id=?id_to_spawn, %error, "failed to spawn pipeline"); + spawn_pipeline_failures.push(id_to_spawn.clone()); } } else { - error!(pipeline_id=?pipeline_to_spawn, "failed to spawn pipeline: source not found"); - spawn_pipeline_failures.push(pipeline_to_spawn.clone()); + error!(pipeline_id=?id_to_spawn, "failed to spawn pipeline: source not found"); + spawn_pipeline_failures.push(id_to_spawn.clone()); } } else { error!( "failed to spawn pipeline: index `{}` no longer exists", - pipeline_to_spawn.index_uid + id_to_spawn.index_uid ); - spawn_pipeline_failures.push(pipeline_to_spawn.clone()); + spawn_pipeline_failures.push(id_to_spawn.clone()); } } Ok(spawn_pipeline_failures) @@ -976,7 +995,7 @@ impl Handler for IndexingService { #[derive(Debug)] struct IndexingPipelineDiff { pipelines_to_shutdown: Vec, - pipelines_to_spawn: Vec, + pipelines_to_spawn: Vec, } #[cfg(test)] From c324f92828ec2c0dd851f1ec14013370f9e1e64b Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:57 +0100 Subject: [PATCH 05/11] WIP Metastore and CP updates added but without tests. Rest root missing. --- .../src/control_plane.rs | 42 +++ .../quickwit-control-plane/src/model/mod.rs | 14 + .../src/metastore/control_plane_metastore.rs | 7 +- .../file_backed/file_backed_index/mod.rs | 5 + .../src/metastore/file_backed/mod.rs | 18 +- .../src/metastore/index_metadata/mod.rs | 18 ++ .../quickwit-metastore/src/metastore/mod.rs | 32 ++- .../src/metastore/postgres/metastore.rs | 21 +- .../protos/quickwit/control_plane.proto | 3 + .../protos/quickwit/metastore.proto | 10 +- .../quickwit/quickwit.control_plane.rs | 247 ++++++++++++++++++ .../codegen/quickwit/quickwit.metastore.rs | 246 ++++++++++++++++- quickwit/quickwit-proto/src/getters.rs | 1 + 13 files changed, 653 insertions(+), 11 deletions(-) diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 80eb94a221a..3480cac53c4 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -51,6 +51,7 @@ use quickwit_proto::metastore::{ DeleteShardsRequest, DeleteSourceRequest, EmptyResponse, FindIndexTemplateMatchesRequest, IndexMetadataResponse, IndexTemplateMatch, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, PruneShardsRequest, ToggleSourceRequest, UpdateIndexRequest, + UpdateSourceRequest, }; use quickwit_proto::types::{IndexId, IndexUid, NodeId, ShardId, SourceId, SourceUid}; use serde::Serialize; @@ -693,6 +694,47 @@ impl Handler for ControlPlane { } } +#[async_trait] +impl Handler for ControlPlane { + type Reply = ControlPlaneResult; + + async fn handle( + &mut self, + request: UpdateSourceRequest, + ctx: &ActorContext, + ) -> Result { + let index_uid: IndexUid = request.index_uid().clone(); + let source_config: SourceConfig = + match serde_utils::from_json_str(&request.source_config_json) { + Ok(source_config) => source_config, + Err(error) => { + return Ok(Err(ControlPlaneError::from(error))); + } + }; + let source_id = source_config.source_id.clone(); + debug!(%index_uid, source_id, "updating source"); + + if let Err(error) = ctx + .protect_future(self.metastore.update_source(request)) + .await + { + return Ok(Err(ControlPlaneError::from(error))); + }; + self.model + .update_source(&index_uid, source_config) + .context("failed to add source")?; + + info!(%index_uid, source_id, "updated source"); + + // TODO: Refine the event. Notify index will have the effect to reload the entire state from + // the metastore. We should update the state of the control plane. + let _rebuild_plan_waiter = self.rebuild_plan_debounced(ctx); + + let response = EmptyResponse {}; + Ok(Ok(response)) + } +} + // This handler is a metastore call proxied through the control plane: we must first forward the // request to the metastore, and then act on the event. #[async_trait] diff --git a/quickwit/quickwit-control-plane/src/model/mod.rs b/quickwit/quickwit-control-plane/src/model/mod.rs index 7a574e97515..34bcb6cbc85 100644 --- a/quickwit/quickwit-control-plane/src/model/mod.rs +++ b/quickwit/quickwit-control-plane/src/model/mod.rs @@ -253,6 +253,20 @@ impl ControlPlaneModel { Ok(()) } + pub(crate) fn update_source( + &mut self, + index_uid: &IndexUid, + source_config: SourceConfig, + ) -> ControlPlaneResult<()> { + let index_metadata = self.index_table.get_mut(index_uid).ok_or_else(|| { + MetastoreError::NotFound(EntityKind::Index { + index_id: index_uid.to_string(), + }) + })?; + index_metadata.update_source(source_config)?; + Ok(()) + } + pub(crate) fn delete_source(&mut self, source_uid: &SourceUid) { // Removing shards from shard table. self.shard_table diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 5d98fcb2faa..4c4ff6cda11 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -36,7 +36,7 @@ use quickwit_proto::metastore::{ ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreResult, MetastoreService, MetastoreServiceClient, MetastoreServiceStream, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, - ToggleSourceRequest, UpdateIndexRequest, UpdateSplitsDeleteOpstampRequest, + ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; @@ -105,6 +105,11 @@ impl MetastoreService for ControlPlaneMetastore { Ok(response) } + async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { + let response = self.control_plane.update_source(request).await?; + Ok(response) + } + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let response = self.control_plane.clone().toggle_source(request).await?; Ok(response) 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 e4d6799cefa..d044ac80642 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 @@ -517,6 +517,11 @@ impl FileBackedIndex { Ok(()) } + /// Updates a source. Returns whether a mutation occurred. + pub(crate) fn update_source(&mut self, source_config: SourceConfig) -> MetastoreResult { + self.metadata.update_source(source_config) + } + /// Enables or disables a source. Returns whether a mutation occurred. pub(crate) fn toggle_source(&mut self, source_id: &str, enable: bool) -> MetastoreResult { self.metadata.toggle_source(source_id, enable) diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs index e4593a8e9d6..6a7423e1540 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs @@ -59,8 +59,8 @@ use quickwit_proto::metastore::{ ListStaleSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, PublishSplitsRequest, ResetSourceCheckpointRequest, - StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSplitsDeleteOpstampRequest, - UpdateSplitsDeleteOpstampResponse, + StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, UpdateSourceRequest, + UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::types::{IndexId, IndexUid}; use quickwit_storage::Storage; @@ -79,7 +79,7 @@ use super::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, IndexesMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsRequestExt, ListSplitsResponseExt, PublishSplitsRequestExt, StageSplitsRequestExt, UpdateIndexRequestExt, - STREAM_SPLITS_CHUNK_SIZE, + UpdateSourceRequestExt, STREAM_SPLITS_CHUNK_SIZE, }; use crate::checkpoint::IndexCheckpointDelta; use crate::{IndexMetadata, ListSplitsQuery, MetastoreServiceExt, Split, SplitState}; @@ -736,6 +736,18 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } + async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid = request.index_uid(); + + self.mutate(index_uid, |index| { + let mutation_occurred = index.update_source(source_config)?; + Ok(MutationOccurred::from(mutation_occurred)) + }) + .await?; + Ok(EmptyResponse {}) + } + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let index_uid = request.index_uid(); diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs index 0d551a26813..d271ab0ca2f 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs @@ -155,6 +155,24 @@ impl IndexMetadata { } } + /// Adds a source to the index. Returns whether a mutation occurred and an + /// error if the source doesn't exist. + pub fn update_source(&mut self, source_config: SourceConfig) -> MetastoreResult { + match self.sources.entry(source_config.source_id.clone()) { + Entry::Occupied(mut entry) => { + if entry.get() == &source_config { + return Ok(false); + } + entry.insert(source_config.clone()); + Ok(true) + } + Entry::Vacant(_) => Err(MetastoreError::NotFound(EntityKind::Source { + index_id: self.index_id().to_string(), + source_id: source_config.source_id, + })), + } + } + pub(crate) fn toggle_source(&mut self, source_id: &str, enable: bool) -> MetastoreResult { let Some(source_config) = self.sources.get_mut(source_id) else { return Err(MetastoreError::NotFound(EntityKind::Source { diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index 719fd603986..6013e9d46d3 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -43,7 +43,7 @@ use quickwit_proto::metastore::{ IndexMetadataFailure, IndexMetadataRequest, IndexMetadataResponse, IndexesMetadataResponse, ListIndexesMetadataResponse, ListSplitsRequest, ListSplitsResponse, MetastoreError, MetastoreResult, MetastoreService, MetastoreServiceClient, MetastoreServiceStream, - PublishSplitsRequest, StageSplitsRequest, UpdateIndexRequest, + PublishSplitsRequest, StageSplitsRequest, UpdateIndexRequest, UpdateSourceRequest, }; use quickwit_proto::types::{IndexUid, NodeId, SplitId}; use time::OffsetDateTime; @@ -426,6 +426,36 @@ impl AddSourceRequestExt for AddSourceRequest { } } +/// Helper trait to build a [`UpdateSourceRequest`] and deserialize its payload. +pub trait UpdateSourceRequestExt { + /// Creates a new [`UpdateSourceRequest`] from a [`SourceConfig`]. + fn try_from_source_config( + index_uid: impl Into, + source_config: &SourceConfig, + ) -> MetastoreResult; + + /// Deserializes the `source_config_json` field of a [`UpdateSourceRequest`] into a + /// [`SourceConfig`]. + fn deserialize_source_config(&self) -> MetastoreResult; +} + +impl UpdateSourceRequestExt for UpdateSourceRequest { + fn try_from_source_config( + index_uid: impl Into, + source_config: &SourceConfig, + ) -> MetastoreResult { + let source_config_json = serde_utils::to_json_str(&source_config)?; + let request = Self { + index_uid: Some(index_uid.into()), + source_config_json, + }; + Ok(request) + } + + fn deserialize_source_config(&self) -> MetastoreResult { + serde_utils::from_json_str(&self.source_config_json) + } +} /// Helper trait to build a [`DeleteTask`] and deserialize its payload. pub trait StageSplitsRequestExt { /// Creates a new [`StageSplitsRequest`] from a [`SplitMetadata`]. diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index c2a3b263514..40f07d03781 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -47,7 +47,8 @@ use quickwit_proto::metastore::{ MetastoreResult, MetastoreService, MetastoreServiceStream, OpenShardSubrequest, OpenShardSubresponse, OpenShardsRequest, OpenShardsResponse, PruneShardsRequest, PublishSplitsRequest, ResetSourceCheckpointRequest, StageSplitsRequest, ToggleSourceRequest, - UpdateIndexRequest, UpdateSplitsDeleteOpstampRequest, UpdateSplitsDeleteOpstampResponse, + UpdateIndexRequest, UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest, + UpdateSplitsDeleteOpstampResponse, }; use quickwit_proto::types::{IndexId, IndexUid, Position, PublishToken, ShardId, SourceId}; use sea_query::{Alias, Asterisk, Expr, Func, PostgresQueryBuilder, Query, UnionType}; @@ -73,7 +74,8 @@ use crate::file_backed::MutationOccurred; use crate::metastore::postgres::model::Shards; use crate::metastore::postgres::utils::split_maturity_timestamp; use crate::metastore::{ - use_shard_api, IndexesMetadataResponseExt, PublishSplitsRequestExt, STREAM_SPLITS_CHUNK_SIZE, + use_shard_api, IndexesMetadataResponseExt, PublishSplitsRequestExt, UpdateSourceRequestExt, + STREAM_SPLITS_CHUNK_SIZE, }; use crate::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadata, IndexMetadataResponseExt, @@ -1080,6 +1082,21 @@ impl MetastoreService for PostgresqlMetastore { Ok(EmptyResponse {}) } + #[instrument(skip(self))] + async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { + let source_config = request.deserialize_source_config()?; + let index_uid: IndexUid = request.index_uid().clone(); + run_with_tx!(self.connection_pool, tx, "add source", { + mutate_index_metadata::(tx, index_uid, |index_metadata| { + let mutation_occurred = index_metadata.update_source(source_config)?; + Ok(MutationOccurred::from(mutation_occurred)) + }) + .await?; + Ok(()) + })?; + Ok(EmptyResponse {}) + } + #[instrument(skip(self))] async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); diff --git a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto index 285732e2669..109fb7eec03 100644 --- a/quickwit/quickwit-proto/protos/quickwit/control_plane.proto +++ b/quickwit/quickwit-proto/protos/quickwit/control_plane.proto @@ -54,6 +54,9 @@ service ControlPlaneService { // Adds a source to an index. rpc AddSource(quickwit.metastore.AddSourceRequest) returns (quickwit.metastore.EmptyResponse); + // Update a source. + rpc UpdateSource(quickwit.metastore.UpdateSourceRequest) returns (quickwit.metastore.EmptyResponse); + // Enables or disables a source. rpc ToggleSource(quickwit.metastore.ToggleSourceRequest) returns (quickwit.metastore.EmptyResponse); diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index 8ae7a8a4400..a61f23bbe94 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -127,9 +127,12 @@ service MetastoreService { // Deletes splits. rpc DeleteSplits(DeleteSplitsRequest) returns (EmptyResponse); - // Adds source. + // Adds a source. rpc AddSource(AddSourceRequest) returns (EmptyResponse); + // Update a source. + rpc UpdateSource(UpdateSourceRequest) returns (EmptyResponse); + // Toggles source. rpc ToggleSource(ToggleSourceRequest) returns (EmptyResponse); @@ -324,6 +327,11 @@ message AddSourceRequest { string source_config_json = 2; } +message UpdateSourceRequest { + quickwit.common.IndexUid index_uid = 1; + string source_config_json = 2; +} + message ToggleSourceRequest { quickwit.common.IndexUid index_uid = 1; string source_id = 2; diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs index c14ef724de0..c747f37c59d 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -154,6 +154,11 @@ pub trait ControlPlaneService: std::fmt::Debug + Send + Sync + 'static { &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult; + /// Update a source. + async fn update_source( + &self, + request: super::metastore::UpdateSourceRequest, + ) -> crate::control_plane::ControlPlaneResult; /// Enables or disables a source. async fn toggle_source( &self, @@ -300,6 +305,12 @@ impl ControlPlaneService for ControlPlaneServiceClient { ) -> crate::control_plane::ControlPlaneResult { self.inner.0.add_source(request).await } + async fn update_source( + &self, + request: super::metastore::UpdateSourceRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner.0.update_source(request).await + } async fn toggle_source( &self, request: super::metastore::ToggleSourceRequest, @@ -372,6 +383,14 @@ pub mod mock_control_plane_service { > { self.inner.lock().await.add_source(request).await } + async fn update_source( + &self, + request: super::super::metastore::UpdateSourceRequest, + ) -> crate::control_plane::ControlPlaneResult< + super::super::metastore::EmptyResponse, + > { + self.inner.lock().await.update_source(request).await + } async fn toggle_source( &self, request: super::super::metastore::ToggleSourceRequest, @@ -483,6 +502,23 @@ for InnerControlPlaneServiceClient { Box::pin(fut) } } +impl tower::Service +for InnerControlPlaneServiceClient { + type Response = super::metastore::EmptyResponse; + type Error = crate::control_plane::ControlPlaneError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: super::metastore::UpdateSourceRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.update_source(request).await }; + Box::pin(fut) + } +} impl tower::Service for InnerControlPlaneServiceClient { type Response = super::metastore::EmptyResponse; @@ -591,6 +627,11 @@ struct ControlPlaneServiceTowerServiceStack { super::metastore::EmptyResponse, crate::control_plane::ControlPlaneError, >, + update_source_svc: quickwit_common::tower::BoxService< + super::metastore::UpdateSourceRequest, + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, + >, toggle_source_svc: quickwit_common::tower::BoxService< super::metastore::ToggleSourceRequest, super::metastore::EmptyResponse, @@ -647,6 +688,12 @@ impl ControlPlaneService for ControlPlaneServiceTowerServiceStack { ) -> crate::control_plane::ControlPlaneResult { self.add_source_svc.clone().ready().await?.call(request).await } + async fn update_source( + &self, + request: super::metastore::UpdateSourceRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.update_source_svc.clone().ready().await?.call(request).await + } async fn toggle_source( &self, request: super::metastore::ToggleSourceRequest, @@ -718,6 +765,16 @@ type AddSourceLayer = quickwit_common::tower::BoxLayer< super::metastore::EmptyResponse, crate::control_plane::ControlPlaneError, >; +type UpdateSourceLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + super::metastore::UpdateSourceRequest, + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, + >, + super::metastore::UpdateSourceRequest, + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, +>; type ToggleSourceLayer = quickwit_common::tower::BoxLayer< quickwit_common::tower::BoxService< super::metastore::ToggleSourceRequest, @@ -774,6 +831,7 @@ pub struct ControlPlaneServiceTowerLayerStack { update_index_layers: Vec, delete_index_layers: Vec, add_source_layers: Vec, + update_source_layers: Vec, toggle_source_layers: Vec, delete_source_layers: Vec, get_or_create_open_shards_layers: Vec, @@ -891,6 +949,33 @@ impl ControlPlaneServiceTowerLayerStack { >>::Service as tower::Service< super::metastore::AddSourceRequest, >>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + super::metastore::UpdateSourceRequest, + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + super::metastore::UpdateSourceRequest, + Response = super::metastore::EmptyResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service< + super::metastore::UpdateSourceRequest, + >>::Future: Send + 'static, L: tower::Layer< quickwit_common::tower::BoxService< super::metastore::ToggleSourceRequest, @@ -1033,6 +1118,8 @@ impl ControlPlaneServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.add_source_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.update_source_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.toggle_source_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.delete_source_layers @@ -1129,6 +1216,27 @@ impl ControlPlaneServiceTowerLayerStack { self.add_source_layers.push(quickwit_common::tower::BoxLayer::new(layer)); self } + pub fn stack_update_source_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + super::metastore::UpdateSourceRequest, + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + super::metastore::UpdateSourceRequest, + Response = super::metastore::EmptyResponse, + Error = crate::control_plane::ControlPlaneError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.update_source_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn stack_toggle_source_layer(mut self, layer: L) -> Self where L: tower::Layer< @@ -1325,6 +1433,14 @@ impl ControlPlaneServiceTowerLayerStack { quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); + let update_source_svc = self + .update_source_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); let toggle_source_svc = self .toggle_source_layers .into_iter() @@ -1371,6 +1487,7 @@ impl ControlPlaneServiceTowerLayerStack { update_index_svc, delete_index_svc, add_source_svc, + update_source_svc, toggle_source_svc, delete_source_svc, get_or_create_open_shards_svc, @@ -1488,6 +1605,15 @@ where crate::control_plane::ControlPlaneError, >, > + + tower::Service< + super::metastore::UpdateSourceRequest, + Response = super::metastore::EmptyResponse, + Error = crate::control_plane::ControlPlaneError, + Future = BoxFuture< + super::metastore::EmptyResponse, + crate::control_plane::ControlPlaneError, + >, + > + tower::Service< super::metastore::ToggleSourceRequest, Response = super::metastore::EmptyResponse, @@ -1562,6 +1688,12 @@ where ) -> crate::control_plane::ControlPlaneResult { self.clone().call(request).await } + async fn update_source( + &self, + request: super::metastore::UpdateSourceRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.clone().call(request).await + } async fn toggle_source( &self, request: super::metastore::ToggleSourceRequest, @@ -1687,6 +1819,20 @@ where super::metastore::AddSourceRequest::rpc_name(), )) } + async fn update_source( + &self, + request: super::metastore::UpdateSourceRequest, + ) -> crate::control_plane::ControlPlaneResult { + self.inner + .clone() + .update_source(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + super::metastore::UpdateSourceRequest::rpc_name(), + )) + } async fn toggle_source( &self, request: super::metastore::ToggleSourceRequest, @@ -1822,6 +1968,17 @@ for ControlPlaneServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(crate::error::grpc_error_to_grpc_status) } + async fn update_source( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .update_source(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } async fn toggle_source( &self, request: tonic::Request, @@ -2089,6 +2246,39 @@ pub mod control_plane_service_grpc_client { ); self.inner.unary(req, path, codec).await } + /// Update a source. + pub async fn update_source( + &mut self, + request: impl tonic::IntoRequest< + super::super::metastore::UpdateSourceRequest, + >, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + > { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.control_plane.ControlPlaneService/UpdateSource", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.control_plane.ControlPlaneService", + "UpdateSource", + ), + ); + self.inner.unary(req, path, codec).await + } /// Enables or disables a source. pub async fn toggle_source( &mut self, @@ -2290,6 +2480,14 @@ pub mod control_plane_service_grpc_server { tonic::Response, tonic::Status, >; + /// Update a source. + async fn update_source( + &self, + request: tonic::Request, + ) -> std::result::Result< + tonic::Response, + tonic::Status, + >; /// Enables or disables a source. async fn toggle_source( &self, @@ -2606,6 +2804,55 @@ pub mod control_plane_service_grpc_server { }; Box::pin(fut) } + "/quickwit.control_plane.ControlPlaneService/UpdateSource" => { + #[allow(non_camel_case_types)] + struct UpdateSourceSvc(pub Arc); + impl< + T: ControlPlaneServiceGrpc, + > tonic::server::UnaryService< + super::super::metastore::UpdateSourceRequest, + > for UpdateSourceSvc { + type Response = super::super::metastore::EmptyResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request< + super::super::metastore::UpdateSourceRequest, + >, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + (*inner).update_source(request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let inner = inner.0; + let method = UpdateSourceSvc(inner); + let codec = tonic::codec::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } "/quickwit.control_plane.ControlPlaneService/ToggleSource" => { #[allow(non_camel_case_types)] struct ToggleSourceSvc(pub Arc); diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 08b12006db3..2256858baf1 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -198,6 +198,15 @@ pub struct AddSourceRequest { #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct UpdateSourceRequest { + #[prost(message, optional, tag = "1")] + pub index_uid: ::core::option::Option, + #[prost(string, tag = "2")] + pub source_config_json: ::prost::alloc::string::String, +} +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct ToggleSourceRequest { #[prost(message, optional, tag = "1")] pub index_uid: ::core::option::Option, @@ -695,6 +704,11 @@ impl RpcName for AddSourceRequest { "add_source" } } +impl RpcName for UpdateSourceRequest { + fn rpc_name() -> &'static str { + "update_source" + } +} impl RpcName for ToggleSourceRequest { fn rpc_name() -> &'static str { "toggle_source" @@ -849,11 +863,16 @@ pub trait MetastoreService: std::fmt::Debug + Send + Sync + 'static { &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult; - /// Adds source. + /// Adds a source. async fn add_source( &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult; + /// Update a source. + async fn update_source( + &self, + request: UpdateSourceRequest, + ) -> crate::metastore::MetastoreResult; /// Toggles source. async fn toggle_source( &self, @@ -1123,6 +1142,12 @@ impl MetastoreService for MetastoreServiceClient { ) -> crate::metastore::MetastoreResult { self.inner.0.add_source(request).await } + async fn update_source( + &self, + request: UpdateSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.0.update_source(request).await + } async fn toggle_source( &self, request: ToggleSourceRequest, @@ -1321,6 +1346,12 @@ pub mod mock_metastore_service { ) -> crate::metastore::MetastoreResult { self.inner.lock().await.add_source(request).await } + async fn update_source( + &self, + request: super::UpdateSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner.lock().await.update_source(request).await + } async fn toggle_source( &self, request: super::ToggleSourceRequest, @@ -1634,6 +1665,22 @@ impl tower::Service for InnerMetastoreServiceClient { Box::pin(fut) } } +impl tower::Service for InnerMetastoreServiceClient { + type Response = EmptyResponse; + type Error = crate::metastore::MetastoreError; + type Future = BoxFuture; + fn poll_ready( + &mut self, + _cx: &mut std::task::Context<'_>, + ) -> std::task::Poll> { + std::task::Poll::Ready(Ok(())) + } + fn call(&mut self, request: UpdateSourceRequest) -> Self::Future { + let svc = self.clone(); + let fut = async move { svc.0.update_source(request).await }; + Box::pin(fut) + } +} impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; @@ -1987,6 +2034,11 @@ struct MetastoreServiceTowerServiceStack { EmptyResponse, crate::metastore::MetastoreError, >, + update_source_svc: quickwit_common::tower::BoxService< + UpdateSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, toggle_source_svc: quickwit_common::tower::BoxService< ToggleSourceRequest, EmptyResponse, @@ -2152,6 +2204,12 @@ impl MetastoreService for MetastoreServiceTowerServiceStack { ) -> crate::metastore::MetastoreResult { self.add_source_svc.clone().ready().await?.call(request).await } + async fn update_source( + &self, + request: UpdateSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.update_source_svc.clone().ready().await?.call(request).await + } async fn toggle_source( &self, request: ToggleSourceRequest, @@ -2387,6 +2445,16 @@ type AddSourceLayer = quickwit_common::tower::BoxLayer< EmptyResponse, crate::metastore::MetastoreError, >; +type UpdateSourceLayer = quickwit_common::tower::BoxLayer< + quickwit_common::tower::BoxService< + UpdateSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + UpdateSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, +>; type ToggleSourceLayer = quickwit_common::tower::BoxLayer< quickwit_common::tower::BoxService< ToggleSourceRequest, @@ -2581,6 +2649,7 @@ pub struct MetastoreServiceTowerLayerStack { mark_splits_for_deletion_layers: Vec, delete_splits_layers: Vec, add_source_layers: Vec, + update_source_layers: Vec, toggle_source_layers: Vec, delete_source_layers: Vec, reset_source_checkpoint_layers: Vec, @@ -2907,6 +2976,31 @@ impl MetastoreServiceTowerLayerStack { crate::metastore::MetastoreError, >, >>::Service as tower::Service>::Future: Send + 'static, + L: tower::Layer< + quickwit_common::tower::BoxService< + UpdateSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Clone + Send + Sync + 'static, + , + >>::Service: tower::Service< + UpdateSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + <, + >>::Service as tower::Service>::Future: Send + 'static, L: tower::Layer< quickwit_common::tower::BoxService< ToggleSourceRequest, @@ -3394,6 +3488,8 @@ impl MetastoreServiceTowerLayerStack { .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.add_source_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); + self.update_source_layers + .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.toggle_source_layers .push(quickwit_common::tower::BoxLayer::new(layer.clone())); self.delete_source_layers @@ -3666,6 +3762,25 @@ impl MetastoreServiceTowerLayerStack { self.add_source_layers.push(quickwit_common::tower::BoxLayer::new(layer)); self } + pub fn stack_update_source_layer(mut self, layer: L) -> Self + where + L: tower::Layer< + quickwit_common::tower::BoxService< + UpdateSourceRequest, + EmptyResponse, + crate::metastore::MetastoreError, + >, + > + Send + Sync + 'static, + L::Service: tower::Service< + UpdateSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + > + Clone + Send + Sync + 'static, + >::Future: Send + 'static, + { + self.update_source_layers.push(quickwit_common::tower::BoxLayer::new(layer)); + self + } pub fn stack_toggle_source_layer(mut self, layer: L) -> Self where L: tower::Layer< @@ -4181,6 +4296,14 @@ impl MetastoreServiceTowerLayerStack { quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); + let update_source_svc = self + .update_source_layers + .into_iter() + .rev() + .fold( + quickwit_common::tower::BoxService::new(inner_client.clone()), + |svc, layer| layer.layer(svc), + ); let toggle_source_svc = self .toggle_source_layers .into_iter() @@ -4339,6 +4462,7 @@ impl MetastoreServiceTowerLayerStack { mark_splits_for_deletion_svc, delete_splits_svc, add_source_svc, + update_source_svc, toggle_source_svc, delete_source_svc, reset_source_checkpoint_svc, @@ -4511,6 +4635,12 @@ where Error = crate::metastore::MetastoreError, Future = BoxFuture, > + + tower::Service< + UpdateSourceRequest, + Response = EmptyResponse, + Error = crate::metastore::MetastoreError, + Future = BoxFuture, + > + tower::Service< ToggleSourceRequest, Response = EmptyResponse, @@ -4707,6 +4837,12 @@ where ) -> crate::metastore::MetastoreResult { self.clone().call(request).await } + async fn update_source( + &self, + request: UpdateSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.clone().call(request).await + } async fn toggle_source( &self, request: ToggleSourceRequest, @@ -5038,6 +5174,20 @@ where AddSourceRequest::rpc_name(), )) } + async fn update_source( + &self, + request: UpdateSourceRequest, + ) -> crate::metastore::MetastoreResult { + self.inner + .clone() + .update_source(request) + .await + .map(|response| response.into_inner()) + .map_err(|status| crate::error::grpc_status_to_service_error( + status, + UpdateSourceRequest::rpc_name(), + )) + } async fn toggle_source( &self, request: ToggleSourceRequest, @@ -5460,6 +5610,17 @@ for MetastoreServiceGrpcServerAdapter { .map(tonic::Response::new) .map_err(crate::error::grpc_error_to_grpc_status) } + async fn update_source( + &self, + request: tonic::Request, + ) -> Result, tonic::Status> { + self.inner + .0 + .update_source(request.into_inner()) + .await + .map(tonic::Response::new) + .map_err(crate::error::grpc_error_to_grpc_status) + } async fn toggle_source( &self, request: tonic::Request, @@ -6104,7 +6265,7 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } - /// Adds source. + /// Adds a source. pub async fn add_source( &mut self, request: impl tonic::IntoRequest, @@ -6129,6 +6290,34 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } + /// Update a source. + pub async fn update_source( + &mut self, + request: impl tonic::IntoRequest, + ) -> std::result::Result, tonic::Status> { + self.inner + .ready() + .await + .map_err(|e| { + tonic::Status::new( + tonic::Code::Unknown, + format!("Service was not ready: {}", e.into()), + ) + })?; + let codec = tonic::codec::ProstCodec::default(); + let path = http::uri::PathAndQuery::from_static( + "/quickwit.metastore.MetastoreService/UpdateSource", + ); + let mut req = request.into_request(); + req.extensions_mut() + .insert( + GrpcMethod::new( + "quickwit.metastore.MetastoreService", + "UpdateSource", + ), + ); + self.inner.unary(req, path, codec).await + } /// Toggles source. pub async fn toggle_source( &mut self, @@ -6758,11 +6947,16 @@ pub mod metastore_service_grpc_server { &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; - /// Adds source. + /// Adds a source. async fn add_source( &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; + /// Update a source. + async fn update_source( + &self, + request: tonic::Request, + ) -> std::result::Result, tonic::Status>; /// Toggles source. async fn toggle_source( &self, @@ -7575,6 +7769,52 @@ pub mod metastore_service_grpc_server { }; Box::pin(fut) } + "/quickwit.metastore.MetastoreService/UpdateSource" => { + #[allow(non_camel_case_types)] + struct UpdateSourceSvc(pub Arc); + impl< + T: MetastoreServiceGrpc, + > tonic::server::UnaryService + for UpdateSourceSvc { + type Response = super::EmptyResponse; + type Future = BoxFuture< + tonic::Response, + tonic::Status, + >; + fn call( + &mut self, + request: tonic::Request, + ) -> Self::Future { + let inner = Arc::clone(&self.0); + let fut = async move { + (*inner).update_source(request).await + }; + Box::pin(fut) + } + } + let accept_compression_encodings = self.accept_compression_encodings; + let send_compression_encodings = self.send_compression_encodings; + let max_decoding_message_size = self.max_decoding_message_size; + let max_encoding_message_size = self.max_encoding_message_size; + let inner = self.inner.clone(); + let fut = async move { + let inner = inner.0; + let method = UpdateSourceSvc(inner); + let codec = tonic::codec::ProstCodec::default(); + let mut grpc = tonic::server::Grpc::new(codec) + .apply_compression_config( + accept_compression_encodings, + send_compression_encodings, + ) + .apply_max_message_size_config( + max_decoding_message_size, + max_encoding_message_size, + ); + let res = grpc.unary(method, req).await; + Ok(res) + }; + Box::pin(fut) + } "/quickwit.metastore.MetastoreService/ToggleSource" => { #[allow(non_camel_case_types)] struct ToggleSourceSvc(pub Arc); diff --git a/quickwit/quickwit-proto/src/getters.rs b/quickwit/quickwit-proto/src/getters.rs index 15ca7235fbe..5c885129a8c 100644 --- a/quickwit/quickwit-proto/src/getters.rs +++ b/quickwit/quickwit-proto/src/getters.rs @@ -140,6 +140,7 @@ generate_getters! { StageSplitsRequest, ToggleSourceRequest, UpdateIndexRequest, + UpdateSourceRequest, UpdateSplitsDeleteOpstampRequest } From 10ad1e3d0c1a2855aeef80e6685cf6aa4117c314 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:57 +0100 Subject: [PATCH 06/11] Add unit tests for metastore and control plane Also include some comment and naming improvments. --- quickwit/quickwit-config/src/lib.rs | 3 +- .../src/control_plane.rs | 107 ++++++++++++++++- .../src/indexing_scheduler/mod.rs | 6 +- .../quickwit-control-plane/src/model/mod.rs | 31 ++++- .../src/actors/indexing_service.rs | 29 +++-- quickwit/quickwit-metastore/src/lib.rs | 2 +- .../src/metastore/index_metadata/mod.rs | 2 +- .../src/metastore/postgres/metastore.rs | 2 +- quickwit/quickwit-metastore/src/tests/mod.rs | 7 ++ .../quickwit-metastore/src/tests/source.rs | 113 +++++++++++++++++- .../protos/quickwit/metastore.proto | 4 +- .../codegen/quickwit/quickwit.metastore.rs | 12 +- 12 files changed, 286 insertions(+), 32 deletions(-) diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index 17f0dea7b29..95cd2cb622c 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -288,7 +288,8 @@ pub trait TestableForRegression: Serialize + DeserializeOwned { fn assert_equality(&self, other: &Self); } -pub fn indexing_params_fingerprint( +/// Return a fingerprint of all parameters that should trigger an indexing pipeline restart. +pub fn indexing_pipeline_params_fingerprint( index_config: &IndexConfig, source_config: &SourceConfig, ) -> u64 { diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 3480cac53c4..58dd12479cc 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -1119,12 +1119,15 @@ async fn watcher_indexers( #[cfg(test)] mod tests { + use std::num::NonZero; use std::sync::Arc; use mockall::Sequence; use quickwit_actors::{AskError, Observe, SupervisorMetrics}; use quickwit_cluster::ClusterChangeStreamFactoryForTest; - use quickwit_config::{IndexConfig, SourceParams, CLI_SOURCE_ID, INGEST_V2_SOURCE_ID}; + use quickwit_config::{ + IndexConfig, KafkaSourceParams, SourceParams, CLI_SOURCE_ID, INGEST_V2_SOURCE_ID, + }; use quickwit_indexing::IndexingService; use quickwit_metastore::{ CreateIndexRequestExt, IndexMetadata, ListIndexesMetadataResponseExt, @@ -1273,7 +1276,8 @@ mod tests { assert_eq!(source_config.source_type(), SourceType::Void); true }) - .returning(|_| Ok(EmptyResponse {})); + .return_once(|_| Ok(EmptyResponse {})); + // the list_indexes_metadata and list_shards calls are made when the control plane starts mock_metastore .expect_list_indexes_metadata() .return_once(move |_| { @@ -1312,6 +1316,105 @@ mod tests { universe.assert_quit().await; } + #[tokio::test] + async fn test_control_plane_update_source() { + let universe = Universe::with_accelerated_time(); + let pipelines_after_update = 3; + let self_node_id: NodeId = "test-node".into(); + let indexer_pool = IndexerPool::default(); + let mut mock_indexer = MockIndexingService::new(); + // call when starting the cp + mock_indexer + .expect_apply_indexing_plan() + .withf(|request| request.indexing_tasks.len() == 1) + .return_once(|_| Ok(ApplyIndexingPlanResponse {})); + // call after the update (3 tasks because 3 pipelines) + mock_indexer + .expect_apply_indexing_plan() + .withf(move |request| request.indexing_tasks.len() == pipelines_after_update) + .return_once(|_| Ok(ApplyIndexingPlanResponse {})); + let indexer = IndexingServiceClient::from_mock(mock_indexer); + let indexer_info = IndexerNodeInfo { + node_id: self_node_id.clone(), + generation_id: 0, + client: indexer, + indexing_tasks: Vec::new(), + indexing_capacity: CpuCapacity::from_cpu_millis(1_000), + }; + indexer_pool.insert(self_node_id.clone(), indexer_info); + + let ingester_pool = IngesterPool::default(); + + let mut index_metadata = IndexMetadata::for_test("test-index", "ram://tata"); + index_metadata + .add_source(SourceConfig::ingest_v2()) + .unwrap(); + + let mut test_source_config = SourceConfig::for_test( + "test-source", + SourceParams::Kafka(KafkaSourceParams { + topic: "test-topic".to_string(), + client_log_level: None, + enable_backfill_mode: false, + client_params: json!({}), + }), + ); + index_metadata + .add_source(test_source_config.clone()) + .unwrap(); + + let mut mock_metastore = MockMetastoreService::new(); + mock_metastore + .expect_update_source() + .withf(move |update_source_request| { + let source_config: SourceConfig = + serde_json::from_str(&update_source_request.source_config_json).unwrap(); + assert_eq!(source_config.source_id, "test-source"); + assert_eq!(source_config.source_type(), SourceType::Kafka); + assert_eq!( + source_config.num_pipelines, + NonZero::new(pipelines_after_update).unwrap() + ); + true + }) + .return_once(|_| Ok(EmptyResponse {})); + // the list_indexes_metadata and list_shards calls are made when the control plane starts + mock_metastore + .expect_list_indexes_metadata() + .return_once(move |_| { + Ok(ListIndexesMetadataResponse::for_test(vec![ + index_metadata.clone() + ])) + }); + mock_metastore + .expect_list_shards() + .return_once(move |_| Ok(ListShardsResponse::default())); + + let cluster_config = ClusterConfig::for_test(); + let cluster_change_stream_factory = ClusterChangeStreamFactoryForTest::default(); + let (control_plane_mailbox, _control_plane_handle, _readiness_rx) = ControlPlane::spawn( + &universe, + cluster_config, + self_node_id, + cluster_change_stream_factory, + indexer_pool, + ingester_pool, + MetastoreServiceClient::from_mock(mock_metastore), + ); + let index_uid: IndexUid = IndexUid::for_test("test-index", 0); + test_source_config.num_pipelines = NonZero::new(pipelines_after_update).unwrap(); + let update_source_request = UpdateSourceRequest { + index_uid: Some(index_uid), + source_config_json: serde_json::to_string(&test_source_config).unwrap(), + }; + control_plane_mailbox + .ask_for_res(update_source_request) + .await + .unwrap(); + + universe.assert_quit().await; + } + #[tokio::test] async fn test_control_plane_toggle_source() { let universe = Universe::with_accelerated_time(); diff --git a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs index a64f1948c12..3d304c03722 100644 --- a/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs +++ b/quickwit/quickwit-control-plane/src/indexing_scheduler/mod.rs @@ -30,7 +30,7 @@ use fnv::{FnvHashMap, FnvHashSet}; use itertools::Itertools; use once_cell::sync::OnceCell; use quickwit_common::pretty::PrettySample; -use quickwit_config::{indexing_params_fingerprint, FileSourceParams, SourceParams}; +use quickwit_config::{indexing_pipeline_params_fingerprint, FileSourceParams, SourceParams}; use quickwit_proto::indexing::{ ApplyIndexingPlanRequest, CpuCapacity, IndexingService, IndexingTask, PIPELINE_FULL_CAPACITY, PIPELINE_THROUGHPUT, @@ -170,7 +170,9 @@ fn get_sources_to_schedule(model: &ControlPlaneModel) -> Vec { } let params_fingerprint = model .index_metadata(&source_uid.index_uid) - .map(|index_meta| indexing_params_fingerprint(&index_meta.index_config, source_config)) + .map(|index_meta| { + indexing_pipeline_params_fingerprint(&index_meta.index_config, source_config) + }) .unwrap_or_default(); match source_config.source_params { SourceParams::File(FileSourceParams::Filepath(_)) diff --git a/quickwit/quickwit-control-plane/src/model/mod.rs b/quickwit/quickwit-control-plane/src/model/mod.rs index 34bcb6cbc85..3fe795c274f 100644 --- a/quickwit/quickwit-control-plane/src/model/mod.rs +++ b/quickwit/quickwit-control-plane/src/model/mod.rs @@ -568,7 +568,7 @@ impl ControlPlaneModel { #[cfg(test)] mod tests { use metastore::EmptyResponse; - use quickwit_config::{SourceConfig, SourceParams, INGEST_V2_SOURCE_ID}; + use quickwit_config::{SourceConfig, SourceParams, TransformConfig, INGEST_V2_SOURCE_ID}; use quickwit_metastore::IndexMetadata; use quickwit_proto::ingest::{Shard, ShardState}; use quickwit_proto::metastore::{ListIndexesMetadataResponse, MockMetastoreService}; @@ -772,6 +772,35 @@ mod tests { ); } + #[test] + fn test_control_plane_model_update_sources() { + let mut model = ControlPlaneModel::default(); + let mut index_metadata = IndexMetadata::for_test("test-index", "ram:///indexes"); + let mut my_source = SourceConfig::for_test("my-source", SourceParams::void()); + index_metadata.add_source(my_source.clone()).unwrap(); + index_metadata + .add_source(SourceConfig::ingest_v2()) + .unwrap(); + let index_uid = index_metadata.index_uid.clone(); + model.add_index(index_metadata.clone()); + + // Update a source + my_source.transform_config = Some(TransformConfig::new("del(.username)".to_string(), None)); + model.update_source(&index_uid, my_source.clone()).unwrap(); + + assert_eq!(model.index_table.len(), 1); + assert_eq!( + model + .index_table + .get(&index_uid) + .unwrap() + .sources + .get("my-source") + .unwrap(), + &my_source + ); + } + #[test] fn test_control_plane_model_delete_index() { let mut model = ControlPlaneModel::default(); diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index b981c3d28ff..bf7d40afa2e 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -36,8 +36,8 @@ use quickwit_common::io::Limiter; use quickwit_common::pubsub::EventBroker; use quickwit_common::{io, temp_dir}; use quickwit_config::{ - build_doc_mapper, indexing_params_fingerprint, IndexConfig, IndexerConfig, SourceConfig, - INGEST_API_SOURCE_ID, + build_doc_mapper, indexing_pipeline_params_fingerprint, IndexConfig, IndexerConfig, + SourceConfig, INGEST_API_SOURCE_ID, }; use quickwit_ingest::{ DropQueueRequest, GetPartitionId, IngestApiService, IngesterPool, ListQueuesRequest, @@ -324,7 +324,8 @@ impl IndexingService { let max_concurrent_split_uploads_merge = (self.max_concurrent_split_uploads - max_concurrent_split_uploads_index).max(1); - let params_fingerprint = indexing_params_fingerprint(&index_config, &source_config); + let params_fingerprint = + indexing_pipeline_params_fingerprint(&index_config, &source_config); if let Some(expected_params_fingerprint) = expected_params_fingerprint { if params_fingerprint != expected_params_fingerprint { warn!( @@ -1220,7 +1221,9 @@ mod tests { #[tokio::test] async fn test_indexing_service_apply_plan() { - const PARAMS_FINGERPRINT: u64 = 3865067856550546352; + const PARAMS_FINGERPRINT_INGEST_API: u64 = 1637744865450232394; + const PARAMS_FINGERPRINT_SOURCE_1: u64 = 1705211905504908791; + const PARAMS_FINGERPRINT_SOURCE_2: u64 = 8706667372658059428; quickwit_common::setup_logging_for_tests(); let transport = ChannelTransport::default(); @@ -1281,14 +1284,14 @@ mod tests { source_id: "test-indexing-service--source-1".to_string(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(0u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, }, IndexingTask { index_uid: Some(metadata.index_uid.clone()), source_id: "test-indexing-service--source-1".to_string(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(1u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, }, ]; indexing_service @@ -1327,28 +1330,28 @@ mod tests { source_id: INGEST_API_SOURCE_ID.to_string(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(3u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_INGEST_API, }, IndexingTask { index_uid: Some(metadata.index_uid.clone()), source_id: "test-indexing-service--source-1".to_string(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(1u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, }, IndexingTask { index_uid: Some(metadata.index_uid.clone()), source_id: "test-indexing-service--source-1".to_string(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(2u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, }, IndexingTask { index_uid: Some(metadata.index_uid.clone()), source_id: source_config_2.source_id.clone(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(4u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_2, }, ]; indexing_service @@ -1389,21 +1392,21 @@ mod tests { source_id: INGEST_API_SOURCE_ID.to_string(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(3u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_INGEST_API, }, IndexingTask { index_uid: Some(metadata.index_uid.clone()), source_id: "test-indexing-service--source-1".to_string(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(1u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, }, IndexingTask { index_uid: Some(metadata.index_uid.clone()), source_id: source_config_2.source_id.clone(), shard_ids: Vec::new(), pipeline_uid: Some(PipelineUid::for_test(4u128)), - params_fingerprint: PARAMS_FINGERPRINT, + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_2, }, ]; indexing_service diff --git a/quickwit/quickwit-metastore/src/lib.rs b/quickwit/quickwit-metastore/src/lib.rs index 78a9bae981c..ffd29584223 100644 --- a/quickwit/quickwit-metastore/src/lib.rs +++ b/quickwit/quickwit-metastore/src/lib.rs @@ -52,7 +52,7 @@ pub use metastore::{ IndexMetadataResponseExt, IndexesMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, ListSplitsResponseExt, MetastoreServiceExt, MetastoreServiceStreamSplitsExt, PublishSplitsRequestExt, StageSplitsRequestExt, - UpdateIndexRequestExt, + UpdateIndexRequestExt, UpdateSourceRequestExt, }; pub use metastore_factory::{MetastoreFactory, UnsupportedMetastore}; pub use metastore_resolver::MetastoreResolver; diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs index d271ab0ca2f..a7859f6f613 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs @@ -163,7 +163,7 @@ impl IndexMetadata { if entry.get() == &source_config { return Ok(false); } - entry.insert(source_config.clone()); + entry.insert(source_config); Ok(true) } Entry::Vacant(_) => Err(MetastoreError::NotFound(EntityKind::Source { diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index 40f07d03781..685e702252e 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -1086,7 +1086,7 @@ impl MetastoreService for PostgresqlMetastore { async fn update_source(&self, request: UpdateSourceRequest) -> MetastoreResult { let source_config = request.deserialize_source_config()?; let index_uid: IndexUid = request.index_uid().clone(); - run_with_tx!(self.connection_pool, tx, "add source", { + run_with_tx!(self.connection_pool, tx, "update source", { mutate_index_metadata::(tx, index_uid, |index_metadata| { let mutation_occurred = index_metadata.update_source(source_config)?; Ok(MutationOccurred::from(mutation_occurred)) diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index e22164a701b..299eb6fde83 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -375,6 +375,13 @@ macro_rules! metastore_test_suite { $crate::tests::source::test_metastore_add_source::<$metastore_type>().await; } + #[tokio::test] + #[serial_test::file_serial] + async fn test_metastore_update_source() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::source::test_metastore_update_source::<$metastore_type>().await; + } + #[tokio::test] #[serial_test::file_serial] async fn test_metastore_toggle_source() { diff --git a/quickwit/quickwit-metastore/src/tests/source.rs b/quickwit/quickwit-metastore/src/tests/source.rs index ba46cbad37e..9af7392caa6 100644 --- a/quickwit/quickwit-metastore/src/tests/source.rs +++ b/quickwit/quickwit-metastore/src/tests/source.rs @@ -20,16 +20,19 @@ use std::num::NonZeroUsize; use quickwit_common::rand::append_random_suffix; -use quickwit_config::{IndexConfig, SourceConfig, SourceInputFormat, SourceParams}; +use quickwit_config::{ + IndexConfig, SourceConfig, SourceInputFormat, SourceParams, TransformConfig, +}; use quickwit_proto::metastore::{ AddSourceRequest, CreateIndexRequest, DeleteSourceRequest, EntityKind, IndexMetadataRequest, MetastoreError, PublishSplitsRequest, ResetSourceCheckpointRequest, SourceType, - StageSplitsRequest, ToggleSourceRequest, + StageSplitsRequest, ToggleSourceRequest, UpdateSourceRequest, }; use quickwit_proto::types::IndexUid; use super::DefaultForTest; use crate::checkpoint::SourceCheckpoint; +use crate::metastore::UpdateSourceRequestExt; use crate::tests::cleanup_index; use crate::{ AddSourceRequestExt, CreateIndexRequestExt, IndexMetadataResponseExt, MetastoreServiceExt, @@ -136,6 +139,112 @@ pub async fn test_metastore_add_source() { + let mut metastore = MetastoreToTest::default_for_test().await; + + let index_id = append_random_suffix("test-add-source"); + 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() + .clone(); + + let source_id = format!("{index_id}--source"); + + let mut source = SourceConfig { + source_id: source_id.to_string(), + num_pipelines: NonZeroUsize::new(1).unwrap(), + enabled: true, + source_params: SourceParams::void(), + transform_config: None, + input_format: SourceInputFormat::Json, + }; + + assert_eq!( + metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap() + .checkpoint + .source_checkpoint(&source_id), + None + ); + + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), &source).unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + + source.transform_config = Some(TransformConfig::new("del(.username)".to_string(), None)); + + // Update the source twice with the same value to validate indempotency + for _ in 0..2 { + let update_source_request = + UpdateSourceRequest::try_from_source_config(index_uid.clone(), &source).unwrap(); + metastore + .update_source(update_source_request) + .await + .unwrap(); + + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + + let sources = &index_metadata.sources; + assert_eq!(sources.len(), 1); + assert!(sources.contains_key(&source_id)); + assert_eq!(sources.get(&source_id).unwrap().source_id, source_id); + assert_eq!( + sources.get(&source_id).unwrap().source_type(), + SourceType::Void + ); + assert_eq!( + sources.get(&source_id).unwrap().transform_config, + Some(TransformConfig::new("del(.username)".to_string(), None)) + ); + assert_eq!( + index_metadata.checkpoint.source_checkpoint(&source_id), + Some(&SourceCheckpoint::default()) + ); + } + + source.source_id = "unknown-src-id".to_string(); + assert!(matches!( + metastore + .update_source( + UpdateSourceRequest::try_from_source_config(index_uid.clone(), &source).unwrap() + ) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Source { .. }) + )); + source.source_id = source_id; + assert!(matches!( + metastore + .add_source( + AddSourceRequest::try_from_source_config( + IndexUid::new_with_random_ulid("index-not-found"), + &source + ) + .unwrap() + ) + .await + .unwrap_err(), + MetastoreError::NotFound(EntityKind::Index { .. }) + )); + + cleanup_index(&mut metastore, index_uid).await; +} + pub async fn test_metastore_toggle_source() { let mut metastore = MetastoreToTest::default_for_test().await; diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index a61f23bbe94..7b739b623be 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -130,10 +130,10 @@ service MetastoreService { // Adds a source. rpc AddSource(AddSourceRequest) returns (EmptyResponse); - // Update a source. + // Updates a source. rpc UpdateSource(UpdateSourceRequest) returns (EmptyResponse); - // Toggles source. + // Toggles (turns on or off) source. rpc ToggleSource(ToggleSourceRequest) returns (EmptyResponse); // Removes source. diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index 2256858baf1..96129b38da1 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -868,12 +868,12 @@ pub trait MetastoreService: std::fmt::Debug + Send + Sync + 'static { &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult; - /// Update a source. + /// Updates a source. async fn update_source( &self, request: UpdateSourceRequest, ) -> crate::metastore::MetastoreResult; - /// Toggles source. + /// Toggles (turns on or off) source. async fn toggle_source( &self, request: ToggleSourceRequest, @@ -6290,7 +6290,7 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } - /// Update a source. + /// Updates a source. pub async fn update_source( &mut self, request: impl tonic::IntoRequest, @@ -6318,7 +6318,7 @@ pub mod metastore_service_grpc_client { ); self.inner.unary(req, path, codec).await } - /// Toggles source. + /// Toggles (turns on or off) source. pub async fn toggle_source( &mut self, request: impl tonic::IntoRequest, @@ -6952,12 +6952,12 @@ pub mod metastore_service_grpc_server { &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; - /// Update a source. + /// Updates a source. async fn update_source( &self, request: tonic::Request, ) -> std::result::Result, tonic::Status>; - /// Toggles source. + /// Toggles (turns on or off) source. async fn toggle_source( &self, request: tonic::Request, From 43b778f8d432fac89105ebf4286556227897d7ac Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:58 +0100 Subject: [PATCH 07/11] Add endpoint --- quickwit/quickwit-config/src/lib.rs | 10 +- .../quickwit-config/src/source_config/mod.rs | 45 +++++++- .../src/source_config/serialize.rs | 28 ++++- .../quickwit-index-management/src/index.rs | 38 ++++++- .../src/metastore/control_plane_metastore.rs | 2 +- .../src/index_api/rest_handler.rs | 101 ++++++++++++++++-- .../src/index_api/source_resource.rs | 65 ++++++++++- 7 files changed, 267 insertions(+), 22 deletions(-) diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index 95cd2cb622c..20f663dc102 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -59,11 +59,11 @@ use serde_json::Value as JsonValue; use siphasher::sip::SipHasher; use source_config::FileSourceParamsForSerde; pub use source_config::{ - load_source_config_from_user_config, FileSourceMessageType, FileSourceNotification, - FileSourceParams, FileSourceSqs, KafkaSourceParams, KinesisSourceParams, PubSubSourceParams, - PulsarSourceAuth, PulsarSourceParams, RegionOrEndpoint, SourceConfig, SourceInputFormat, - SourceParams, TransformConfig, VecSourceParams, VoidSourceParams, CLI_SOURCE_ID, - INGEST_API_SOURCE_ID, INGEST_V2_SOURCE_ID, + load_source_config_from_user_config, load_source_config_update, FileSourceMessageType, + FileSourceNotification, FileSourceParams, FileSourceSqs, KafkaSourceParams, + KinesisSourceParams, PubSubSourceParams, PulsarSourceAuth, PulsarSourceParams, + RegionOrEndpoint, SourceConfig, SourceInputFormat, SourceParams, TransformConfig, + VecSourceParams, VoidSourceParams, CLI_SOURCE_ID, INGEST_API_SOURCE_ID, INGEST_V2_SOURCE_ID, }; use tracing::warn; diff --git a/quickwit/quickwit-config/src/source_config/mod.rs b/quickwit/quickwit-config/src/source_config/mod.rs index d8395ae5fe8..c0b9eab809f 100644 --- a/quickwit/quickwit-config/src/source_config/mod.rs +++ b/quickwit/quickwit-config/src/source_config/mod.rs @@ -33,9 +33,9 @@ use regex::Regex; use serde::de::Error; use serde::{Deserialize, Deserializer, Serialize}; use serde_json::Value as JsonValue; -pub use serialize::load_source_config_from_user_config; // For backward compatibility. use serialize::VersionedSourceConfig; +pub use serialize::{load_source_config_from_user_config, load_source_config_update}; use siphasher::sip::SipHasher; use crate::{disable_ingest_v1, enable_ingest_v2}; @@ -632,6 +632,7 @@ impl TransformConfig { #[cfg(test)] mod tests { + use std::num::NonZero; use std::str::FromStr; use quickwit_common::uri::Uri; @@ -1392,4 +1393,46 @@ mod tests { .unwrap(); assert_eq!(source_config.input_format, SourceInputFormat::PlainText); } + + #[tokio::test] + async fn test_update_kafka_source_config() { + let source_config_filepath = get_source_config_filepath("kafka-source.json"); + let file_content = std::fs::read(&source_config_filepath).unwrap(); + let source_config_uri = Uri::from_str(&source_config_filepath).unwrap(); + let config_format = ConfigFormat::sniff_from_uri(&source_config_uri).unwrap(); + let mut existing_source_config = + load_source_config_from_user_config(config_format, &file_content).unwrap(); + existing_source_config.num_pipelines = NonZero::new(4).unwrap(); + let new_source_config = + load_source_config_update(config_format, &file_content, &existing_source_config) + .unwrap(); + + let expected_source_config = SourceConfig { + source_id: "hdfs-logs-kafka-source".to_string(), + num_pipelines: NonZeroUsize::new(2).unwrap(), + enabled: true, + source_params: SourceParams::Kafka(KafkaSourceParams { + topic: "cloudera-cluster-logs".to_string(), + client_log_level: None, + client_params: json! {{"bootstrap.servers": "localhost:9092"}}, + enable_backfill_mode: false, + }), + transform_config: Some(TransformConfig { + vrl_script: ".message = downcase(string!(.message))".to_string(), + timezone: "local".to_string(), + }), + input_format: SourceInputFormat::Json, + }; + assert_eq!(new_source_config, expected_source_config); + assert_eq!(new_source_config.num_pipelines.get(), 2); + + // the source type cannot be updated + existing_source_config.source_params = SourceParams::Kinesis(KinesisSourceParams { + stream_name: "my-stream".to_string(), + region_or_endpoint: None, + enable_backfill_mode: false, + }); + load_source_config_update(config_format, &file_content, &existing_source_config) + .unwrap_err(); + } } diff --git a/quickwit/quickwit-config/src/source_config/serialize.rs b/quickwit/quickwit-config/src/source_config/serialize.rs index c747387b0f2..e3c3d03f9d1 100644 --- a/quickwit/quickwit-config/src/source_config/serialize.rs +++ b/quickwit/quickwit-config/src/source_config/serialize.rs @@ -19,7 +19,7 @@ use std::num::NonZeroUsize; -use anyhow::bail; +use anyhow::{bail, ensure}; use quickwit_proto::types::SourceId; use serde::{Deserialize, Serialize}; @@ -64,6 +64,32 @@ pub fn load_source_config_from_user_config( source_config_for_serialization.validate_and_build() } +pub fn load_source_config_update( + config_format: ConfigFormat, + config_content: &[u8], + current_source_config: &SourceConfig, +) -> anyhow::Result { + let versioned_source_config: VersionedSourceConfig = config_format.parse(config_content)?; + let source_config_for_serialization: SourceConfigForSerialization = + versioned_source_config.into(); + let new_source_config = source_config_for_serialization.validate_and_build()?; + + ensure!( + current_source_config.source_id == new_source_config.source_id, + "existing `source_id` {} does not match updated `source_id` {}", + current_source_config.source_id, + new_source_config.source_id + ); + + ensure!( + current_source_config.source_type() == new_source_config.source_type(), + "source type cannot be updated, current type: {}", + current_source_config.source_type(), + ); + + Ok(new_source_config) +} + impl SourceConfigForSerialization { /// Checks the validity of the `SourceConfig` as a "deserializable source". /// diff --git a/quickwit/quickwit-index-management/src/index.rs b/quickwit/quickwit-index-management/src/index.rs index 0fe5c77cc2b..401cf10dbfa 100644 --- a/quickwit/quickwit-index-management/src/index.rs +++ b/quickwit/quickwit-index-management/src/index.rs @@ -31,13 +31,13 @@ use quickwit_indexing::check_source_connectivity; use quickwit_metastore::{ AddSourceRequestExt, CreateIndexResponseExt, IndexMetadata, IndexMetadataResponseExt, ListIndexesMetadataResponseExt, ListSplitsQuery, ListSplitsRequestExt, - MetastoreServiceStreamSplitsExt, SplitInfo, SplitMetadata, SplitState, + MetastoreServiceStreamSplitsExt, SplitInfo, SplitMetadata, SplitState, UpdateSourceRequestExt, }; use quickwit_proto::metastore::{ serde_utils, AddSourceRequest, CreateIndexRequest, DeleteIndexRequest, EntityKind, IndexMetadataRequest, ListIndexesMetadataRequest, ListSplitsRequest, MarkSplitsForDeletionRequest, MetastoreError, MetastoreService, MetastoreServiceClient, - ResetSourceCheckpointRequest, + ResetSourceCheckpointRequest, UpdateSourceRequest, }; use quickwit_proto::types::{IndexUid, SplitId}; use quickwit_proto::{ServiceError, ServiceErrorCode}; @@ -481,6 +481,40 @@ impl IndexService { Ok(source) } + /// Updates a source from an index identified by its UID. + pub async fn update_source( + &mut self, + index_uid: IndexUid, + source_config: SourceConfig, + ) -> Result { + let source_id = source_config.source_id.clone(); + check_source_connectivity(&self.storage_resolver, &source_config) + .await + .map_err(IndexServiceError::InvalidConfig)?; + let update_source_request = + UpdateSourceRequest::try_from_source_config(index_uid.clone(), &source_config)?; + self.metastore.update_source(update_source_request).await?; + info!( + "source `{}` successfully updated for index `{}`", + source_id, index_uid.index_id, + ); + let index_metadata_request = IndexMetadataRequest::for_index_id(index_uid.index_id); + let source = self + .metastore + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()? + .sources + .get(&source_id) + .ok_or_else(|| { + IndexServiceError::Internal( + "created source is not in index metadata, this should never happen".to_string(), + ) + })? + .clone(); + Ok(source) + } + pub async fn get_source( &mut self, index_id: &str, diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 4c4ff6cda11..2880831cc60 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -111,7 +111,7 @@ impl MetastoreService for ControlPlaneMetastore { } async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { - let response = self.control_plane.clone().toggle_source(request).await?; + let response = self.control_plane.toggle_source(request).await?; Ok(response) } diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index 25e7d52bb43..d1adb91a3a6 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -37,9 +37,9 @@ use super::index_resource::{ }; use super::source_resource::{ __path_create_source, __path_delete_source, __path_reset_source_checkpoint, - __path_toggle_source, create_source_handler, delete_source_handler, get_source_handler, - get_source_shards_handler, reset_source_checkpoint_handler, toggle_source_handler, - ToggleSource, + __path_toggle_source, __path_update_source, create_source_handler, delete_source_handler, + get_source_handler, get_source_shards_handler, reset_source_checkpoint_handler, + toggle_source_handler, update_source_handler, ToggleSource, }; use super::split_resource::{ __path_list_splits, __path_mark_splits_for_deletion, list_splits_handler, @@ -62,6 +62,7 @@ use crate::simple_list::from_simple_list; describe_index, mark_splits_for_deletion, create_source, + update_source, reset_source_checkpoint, toggle_source, delete_source, @@ -107,6 +108,7 @@ pub fn index_management_handlers( .or(reset_source_checkpoint_handler(index_service.metastore())) .or(toggle_source_handler(index_service.metastore())) .or(create_source_handler(index_service.clone())) + .or(update_source_handler(index_service.clone())) .or(get_source_handler(index_service.metastore())) .or(delete_source_handler(index_service.metastore())) .or(get_source_shards_handler(index_service.metastore())) @@ -1109,6 +1111,92 @@ mod tests { } } + #[tokio::test] + async fn test_update_source() { + let metastore = metastore_for_test(); + let index_service = IndexService::new(metastore.clone(), StorageResolver::unconfigured()); + let mut node_config = NodeConfig::for_test(); + node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); + let index_management_handler = + super::index_management_handlers(index_service, Arc::new(node_config)); + let resp = warp::test::request() + .path("/indexes") + .method("POST") + .json(&true) + .body(r#"{"version": "0.7", "index_id": "hdfs-logs", "doc_mapping": {"field_mappings":[{"name": "timestamp", "type": "i64", "fast": true, "indexed": true}]}}"#) + .reply(&index_management_handler) + .await; + assert_eq!(resp.status(), 200); + let resp_json: serde_json::Value = serde_json::from_slice(resp.body()).unwrap(); + let expected_response_json = serde_json::json!({ + "index_config": { + "index_id": "hdfs-logs", + "index_uri": "file:///default-index-root-uri/hdfs-logs", + } + }); + assert_json_include!(actual: resp_json, expected: expected_response_json); + + // Create source. + let source_config_body = r#"{"version": "0.7", "source_id": "vec-source", "source_type": "vec", "params": {"docs": [], "batch_num_docs": 10}}"#; + let resp = warp::test::request() + .path("/indexes/hdfs-logs/sources") + .method("POST") + .json(&true) + .body(source_config_body) + .reply(&index_management_handler) + .await; + assert_eq!(resp.status(), 200); + + // Update the source. + let update_source_config_body = r#"{"version": "0.7", "source_id": "vec-source", "source_type": "vec", "params": {"docs": [], "batch_num_docs": 20}}"#; + let resp = warp::test::request() + .path("/indexes/hdfs-logs/sources/vec-source") + .method("PUT") + .json(&true) + .body(update_source_config_body) + .reply(&index_management_handler) + .await; + assert_eq!(resp.status(), 200); + // Check that the source has been updated. + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert!(index_metadata.sources.contains_key("vec-source")); + let source_config = index_metadata.sources.get("vec-source").unwrap(); + assert_eq!(source_config.source_type(), SourceType::Vec); + assert_eq!( + source_config.source_params, + SourceParams::Vec(VecSourceParams { + docs: Vec::new(), + batch_num_docs: 20, + partition: "".to_string(), + }) + ); + + // Update the source with a different source_id (forbidden) + let update_source_config_body = r#"{"version": "0.7", "source_id": "other-source-id", "source_type": "vec", "params": {"docs": [], "batch_num_docs": 20}}"#; + let resp = warp::test::request() + .path("/indexes/hdfs-logs/sources/vec-source") + .method("PUT") + .json(&true) + .body(update_source_config_body) + .reply(&index_management_handler) + .await; + assert_eq!(resp.status(), 400); + // Check that the source hasn't been updated. + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert!(index_metadata.sources.contains_key("vec-source")); + assert!(!index_metadata.sources.contains_key("other-source-id")); + } + #[tokio::test] async fn test_delete_non_existing_source() { let mut mock_metastore = MockMetastoreService::new(); @@ -1244,13 +1332,6 @@ mod tests { let index_management_handler = super::index_management_handlers(index_service, Arc::new(NodeConfig::for_test())) .recover(recover_fn); - // Check server returns 405 if sources root path is used. - let resp = warp::test::request() - .path("/indexes/quickwit-demo-index/sources/source-to-toggle") - .method("PUT") - .reply(&index_management_handler) - .await; - assert_eq!(resp.status(), 405); let resp = warp::test::request() .path("/indexes/quickwit-demo-index/sources/source-to-toggle/toggle") .method("PUT") diff --git a/quickwit/quickwit-serve/src/index_api/source_resource.rs b/quickwit/quickwit-serve/src/index_api/source_resource.rs index 2ece76047b3..b4d9526f5df 100644 --- a/quickwit/quickwit-serve/src/index_api/source_resource.rs +++ b/quickwit/quickwit-serve/src/index_api/source_resource.rs @@ -19,8 +19,8 @@ use bytes::Bytes; use quickwit_config::{ - load_source_config_from_user_config, ConfigFormat, FileSourceParams, SourceConfig, - SourceParams, CLI_SOURCE_ID, INGEST_API_SOURCE_ID, + load_source_config_from_user_config, load_source_config_update, ConfigFormat, FileSourceParams, + SourceConfig, SourceParams, CLI_SOURCE_ID, INGEST_API_SOURCE_ID, }; use quickwit_index_management::{IndexService, IndexServiceError}; use quickwit_metastore::IndexMetadataResponseExt; @@ -100,6 +100,67 @@ pub async fn create_source( index_service.add_source(index_uid, source_config).await } +pub fn update_source_handler( + index_service: IndexService, +) -> impl Filter + Clone { + warp::path!("indexes" / String / "sources" / String) + .and(warp::put()) + .and(extract_config_format()) + .and(warp::body::content_length_limit(1024 * 1024)) + .and(warp::filters::body::bytes()) + .and(with_arg(index_service)) + .then(update_source) + .map(log_failure("failed to update source")) + .and(extract_format_from_qs()) + .map(into_rest_api_response) + .boxed() +} + +#[utoipa::path( + put, + tag = "Sources", + path = "/indexes/{index_id}/sources/{source_id}", + request_body = VersionedSourceConfig, + responses( + // We return `VersionedSourceConfig` as it's the serialized model view. + (status = 200, description = "Successfully updated source.", body = VersionedSourceConfig) + ), + params( + ("index_id" = String, Path, description = "The index ID to create a source for."), + ("source_id" = String, Path, description = "The source ID to update."), + ) +)] +/// Updates Source. +pub async fn update_source( + index_id: IndexId, + source_id: SourceId, + config_format: ConfigFormat, + source_config_bytes: Bytes, + mut index_service: IndexService, +) -> Result { + let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); + let mut current_index_metadata = index_service + .metastore() + .index_metadata(index_metadata_request) + .await? + .deserialize_index_metadata()?; + let current_source_config = current_index_metadata.sources.remove(&source_id).ok_or({ + MetastoreError::NotFound(EntityKind::Source { + index_id: index_id.to_string(), + source_id, + }) + })?; + + let new_source_config: SourceConfig = + load_source_config_update(config_format, &source_config_bytes, ¤t_source_config) + .map_err(IndexServiceError::InvalidConfig)?; + + info!(index_id = %index_id, source_id = %new_source_config.source_id, "update-source"); + index_service + .update_source(current_index_metadata.index_uid, new_source_config) + .await +} + pub fn get_source_handler( metastore: MetastoreServiceClient, ) -> impl Filter + Clone { From 1406f0a06967b99b63e37dc9b47eef6b1b0d04ef Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:58 +0100 Subject: [PATCH 08/11] Add cli --- docs/reference/cli.md | 21 +++++ quickwit/quickwit-cli/src/source.rs | 88 +++++++++++++++++++ .../quickwit-rest-client/src/rest_client.rs | 43 +++++++++ 3 files changed, 152 insertions(+) diff --git a/docs/reference/cli.md b/docs/reference/cli.md index 543cc4d69fb..adde7e9e7d8 100644 --- a/docs/reference/cli.md +++ b/docs/reference/cli.md @@ -493,6 +493,27 @@ quickwit source create |-----------------|-------------| | `--index` | ID of the target index | | `--source-config` | Path to source config file. Please, refer to the documentation for more details. | +### source update + +Update an existing source. +`quickwit source update [args]` + +*Synopsis* + +```bash +quickwit source update + --index + --source + --source-config +``` + +*Options* + +| Option | Description | +|-----------------|-------------| +| `--index` | ID of the target index | +| `--source` | ID of the source | +| `--source-config` | Path to source config file. Please, refer to the documentation for more details. | ### source enable Enables a source for an index. diff --git a/quickwit/quickwit-cli/src/source.rs b/quickwit/quickwit-cli/src/source.rs index ee90689ca94..bd7173552fb 100644 --- a/quickwit/quickwit-cli/src/source.rs +++ b/quickwit/quickwit-cli/src/source.rs @@ -50,6 +50,20 @@ pub fn build_source_command() -> Command { .required(true), ]) ) + .subcommand( + Command::new("update") + .about("Update an existing source.") + .args(&[ + arg!(--index "ID of the target index") + .display_order(1) + .required(true), + arg!(--source "ID of the source") + .display_order(2) + .required(true), + arg!(--"source-config" "Path to source config file. Please, refer to the documentation for more details.") + .required(true), + ]) + ) .subcommand( Command::new("enable") .about("Enables a source for an index.") @@ -147,6 +161,14 @@ pub struct CreateSourceArgs { pub source_config_uri: Uri, } +#[derive(Debug, Eq, PartialEq)] +pub struct UpdateSourceArgs { + pub client_args: ClientArgs, + pub index_id: IndexId, + pub source_id: SourceId, + pub source_config_uri: Uri, +} + #[derive(Debug, Eq, PartialEq)] pub struct ToggleSourceArgs { pub client_args: ClientArgs, @@ -187,6 +209,7 @@ pub struct ResetCheckpointArgs { #[derive(Debug, Eq, PartialEq)] pub enum SourceCliCommand { CreateSource(CreateSourceArgs), + UpdateSource(UpdateSourceArgs), ToggleSource(ToggleSourceArgs), DeleteSource(DeleteSourceArgs), DescribeSource(DescribeSourceArgs), @@ -198,6 +221,7 @@ impl SourceCliCommand { pub async fn execute(self) -> anyhow::Result<()> { match self { Self::CreateSource(args) => create_source_cli(args).await, + Self::UpdateSource(args) => update_source_cli(args).await, Self::ToggleSource(args) => toggle_source_cli(args).await, Self::DeleteSource(args) => delete_source_cli(args).await, Self::DescribeSource(args) => describe_source_cli(args).await, @@ -212,6 +236,7 @@ impl SourceCliCommand { .context("failed to parse source subcommand")?; match subcommand.as_str() { "create" => Self::parse_create_args(submatches).map(Self::CreateSource), + "update" => Self::parse_update_args(submatches).map(Self::UpdateSource), "enable" => { Self::parse_toggle_source_args(&subcommand, submatches).map(Self::ToggleSource) } @@ -244,6 +269,26 @@ impl SourceCliCommand { }) } + fn parse_update_args(mut matches: ArgMatches) -> anyhow::Result { + let client_args = ClientArgs::parse(&mut matches)?; + let index_id = matches + .remove_one::("index") + .expect("`index` should be a required arg."); + let source_id = matches + .remove_one::("source") + .expect("`source` should be a required arg."); + let source_config_uri = matches + .remove_one::("source-config") + .map(|uri_str| Uri::from_str(&uri_str)) + .expect("`source-config` should be a required arg.")?; + Ok(UpdateSourceArgs { + client_args, + index_id, + source_id, + source_config_uri, + }) + } + fn parse_toggle_source_args( subcommand: &str, mut matches: ArgMatches, @@ -342,6 +387,23 @@ async fn create_source_cli(args: CreateSourceArgs) -> anyhow::Result<()> { Ok(()) } +async fn update_source_cli(args: UpdateSourceArgs) -> anyhow::Result<()> { + debug!(args=?args, "update-source"); + println!("❯ Updating source..."); + let storage_resolver = StorageResolver::unconfigured(); + let source_config_content = load_file(&storage_resolver, &args.source_config_uri).await?; + let source_config_str: &str = std::str::from_utf8(&source_config_content) + .with_context(|| format!("source config is not utf-8: {}", args.source_config_uri))?; + let config_format = ConfigFormat::sniff_from_uri(&args.source_config_uri)?; + let qw_client = args.client_args.client(); + qw_client + .sources(&args.index_id) + .update(&args.source_id, source_config_str, config_format) + .await?; + println!("{} Source successfully updated.", "✔".color(GREEN_COLOR)); + Ok(()) +} + async fn toggle_source_cli(args: ToggleSourceArgs) -> anyhow::Result<()> { debug!(args=?args, "toggle-source"); println!("❯ Toggling source..."); @@ -604,6 +666,32 @@ mod tests { assert_eq!(command, expected_command); } + #[test] + fn test_parse_update_source_args() { + let app = build_cli().no_binary_name(true); + let matches = app + .try_get_matches_from(vec![ + "source", + "update", + "--index", + "hdfs-logs", + "--source", + "kafka-foo", + "--source-config", + "/source-conf.yaml", + ]) + .unwrap(); + let command = CliCommand::parse_cli_args(matches).unwrap(); + let expected_command = + CliCommand::Source(SourceCliCommand::UpdateSource(UpdateSourceArgs { + client_args: ClientArgs::default(), + index_id: "hdfs-logs".to_string(), + source_id: "kafka-foo".to_string(), + source_config_uri: Uri::from_str("file:///source-conf.yaml").unwrap(), + })); + assert_eq!(command, expected_command); + } + #[test] fn test_parse_toggle_source_args() { { diff --git a/quickwit/quickwit-rest-client/src/rest_client.rs b/quickwit/quickwit-rest-client/src/rest_client.rs index ff2f952275d..c60b93c9872 100644 --- a/quickwit/quickwit-rest-client/src/rest_client.rs +++ b/quickwit/quickwit-rest-client/src/rest_client.rs @@ -540,6 +540,30 @@ impl<'a> SourceClient<'a> { Ok(source_config) } + pub async fn update( + &self, + source_id: &str, + source_config_input: impl AsRef<[u8]>, + config_format: ConfigFormat, + ) -> Result { + let header_map = header_from_config_format(config_format); + let source_config_bytes = Bytes::copy_from_slice(source_config_input.as_ref()); + let path = format!("{}/{source_id}", self.sources_root_url()); + let response = self + .transport + .send::<()>( + Method::PUT, + &path, + Some(header_map), + None, + Some(source_config_bytes), + self.timeout, + ) + .await?; + let source_config = response.deserialize().await?; + Ok(source_config) + } + pub async fn get(&self, source_id: &str) -> Result { let path = format!("{}/{source_id}", self.sources_root_url()); let response = self @@ -1133,6 +1157,25 @@ mod test { source_config ); + // PUT update source with yaml + Mock::given(method("PUT")) + .and(path("/api/v1/indexes/my-index/sources/my-source-1")) + .and(header(CONTENT_TYPE.as_str(), "application/yaml")) + .respond_with( + ResponseTemplate::new(StatusCode::OK).set_body_json(source_config.clone()), + ) + .up_to_n_times(1) + .mount(&mock_server) + .await; + assert_eq!( + qw_client + .sources("my-index") + .update("my-source-1", "", ConfigFormat::Yaml) + .await + .unwrap(), + source_config + ); + // GET sources Mock::given(method("GET")) .and(path("/api/v1/indexes/my-index/sources")) From ce61a97e144c4f243394f208e5cc7cde0308065a Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Tue, 21 Jan 2025 16:46:58 +0100 Subject: [PATCH 09/11] Add integration test --- .../source_config/invalid-void-source.json | 8 -- .../src/tests/sqs_tests.rs | 117 ++++++++++++++++++ 2 files changed, 117 insertions(+), 8 deletions(-) delete mode 100644 quickwit/quickwit-config/resources/tests/source_config/invalid-void-source.json diff --git a/quickwit/quickwit-config/resources/tests/source_config/invalid-void-source.json b/quickwit/quickwit-config/resources/tests/source_config/invalid-void-source.json deleted file mode 100644 index e16da6a0716..00000000000 --- a/quickwit/quickwit-config/resources/tests/source_config/invalid-void-source.json +++ /dev/null @@ -1,8 +0,0 @@ -{ - "version": "0.7", - "source_id": "hdfs-logs-void-source", - "desired_num_pipelines": 0, - "max_num_pipelines_per_indexer": 1, - "source_type": "void", - "params": {} -} diff --git a/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs b/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs index 6999c757cea..870a2875bd6 100644 --- a/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/sqs_tests.rs @@ -246,3 +246,120 @@ async fn test_sqs_garbage_collect() { sandbox.shutdown().await.unwrap(); } + +// this source update test is done here because SQS is the only long running +// configurable source for which we have integration tests set up. +#[tokio::test] +async fn test_update_source_multi_node_cluster() { + quickwit_common::setup_logging_for_tests(); + let index_id = "test-update-source-cluster"; + let sqs_client = sqs_test_helpers::get_localstack_sqs_client().await.unwrap(); + let queue_url = sqs_test_helpers::create_queue(&sqs_client, "test-update-source-cluster").await; + + let sandbox = ClusterSandboxBuilder::default() + .add_node([QuickwitService::Searcher]) + .add_node([QuickwitService::Metastore]) + .add_node([QuickwitService::Indexer]) + .add_node([QuickwitService::ControlPlane]) + .add_node([QuickwitService::Janitor]) + .build_and_start() + .await; + + { + // Wait for indexer to fully start. + // The starting time is a bit long for a cluster. + tokio::time::sleep(Duration::from_secs(3)).await; + let indexing_service_counters = sandbox + .rest_client(QuickwitService::Indexer) + .node_stats() + .indexing() + .await + .unwrap(); + assert_eq!(indexing_service_counters.num_running_pipelines, 0); + } + + // Create an index + let index_config = format!( + r#" + version: 0.8 + index_id: {} + doc_mapping: + field_mappings: + - name: body + type: text + indexing_settings: + commit_timeout_secs: 1 + "#, + index_id + ); + sandbox + .rest_client(QuickwitService::Indexer) + .indexes() + .create(index_config, ConfigFormat::Yaml, false) + .await + .unwrap(); + + // Wait until indexing pipelines are started + sandbox.wait_for_indexing_pipelines(1).await.unwrap(); + + // create an SQS source with 1 pipeline + let source_id: &str = "test-update-source-cluster"; + let source_config_input = format!( + r#" + version: 0.7 + source_id: {} + desired_num_pipelines: 1 + max_num_pipelines_per_indexer: 1 + source_type: file + params: + notifications: + - type: sqs + queue_url: {} + message_type: raw_uri + deduplication_window_max_messages: 5 + deduplication_cleanup_interval_secs: 3 + input_format: plain_text + "#, + source_id, queue_url + ); + sandbox + .rest_client(QuickwitService::Indexer) + .sources(index_id) + .create(source_config_input, ConfigFormat::Yaml) + .await + .unwrap(); + + // Wait until the SQS indexing pipeline is also started + sandbox.wait_for_indexing_pipelines(2).await.unwrap(); + + // increase the number of pipelines to 3 + let source_config_input = format!( + r#" + version: 0.7 + source_id: {} + desired_num_pipelines: 3 + max_num_pipelines_per_indexer: 3 + source_type: file + params: + notifications: + - type: sqs + queue_url: {} + message_type: raw_uri + deduplication_window_max_messages: 5 + deduplication_cleanup_interval_secs: 3 + input_format: plain_text + "#, + source_id, queue_url + ); + sandbox + .rest_client(QuickwitService::Metastore) + .sources(index_id) + .update(source_id, source_config_input, ConfigFormat::Yaml) + .await + .unwrap(); + + // Wait until the SQS indexing pipeline is also started + sandbox.wait_for_indexing_pipelines(4).await.unwrap(); + + sandbox.shutdown().await.unwrap(); +} From ea13a18f4fe5e4d0c3034e93c249f4d6d22e9a4e Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Wed, 22 Jan 2025 16:33:37 +0100 Subject: [PATCH 10/11] Deeper update validation and abort on fingerprint missmatch --- docs/operating/upgrades.md | 2 +- quickwit/quickwit-cli/src/source.rs | 2 +- .../quickwit-config/src/index_config/mod.rs | 7 +- quickwit/quickwit-config/src/lib.rs | 3 +- .../quickwit-config/src/source_config/mod.rs | 197 +++++++--- .../src/source_config/serialize.rs | 13 +- .../quickwit-index-management/src/index.rs | 4 +- .../src/actors/indexing_service.rs | 366 ++++++++++-------- 8 files changed, 381 insertions(+), 213 deletions(-) diff --git a/docs/operating/upgrades.md b/docs/operating/upgrades.md index 96dc8e577dc..ec60855baeb 100644 --- a/docs/operating/upgrades.md +++ b/docs/operating/upgrades.md @@ -23,4 +23,4 @@ No migration is done if `otel-traces-v0_7` already exists. If you want `service_ Quickwit 0.9 introduces a new ingestion service to to power the ingest and bulk APIs (v2). The new ingest is enabled and used by default, even though the legacy one (v1) remains enabled to finish indexing residual data in the legacy write ahead logs. Note that `ingest_api.max_queue_disk_usage` is enforced on both ingest versions separately, which means that the cumulated disk usage might be up to twice this limit. -The control plane should be upgraded first in order to enable the new ingest source (v2) on all existing indexes. Ingested data into previously existing indexes on upgraded indexer nodes will not be picked by the indexing pipelines until the control plane is upgraded. +The control plane should be upgraded first in order to enable the new ingest source (v2) on all existing indexes. Ingested data into previously existing indexes on upgraded indexer nodes will not be picked by the indexing pipelines until the control plane is upgraded. Because the indexing plan is computed differently in 0.9, all pipelines will be restarted when upgrading the control plane. If possible, we recommend avoiding rolling upgrades for indexers. Instead, scale down the number of indexers to zero first, then upgrade the control plane and finally scale the upgraded indexers back up. diff --git a/quickwit/quickwit-cli/src/source.rs b/quickwit/quickwit-cli/src/source.rs index bd7173552fb..1996ed09558 100644 --- a/quickwit/quickwit-cli/src/source.rs +++ b/quickwit/quickwit-cli/src/source.rs @@ -52,7 +52,7 @@ pub fn build_source_command() -> Command { ) .subcommand( Command::new("update") - .about("Update an existing source.") + .about("Updates an existing source.") .args(&[ arg!(--index "ID of the target index") .display_order(1) diff --git a/quickwit/quickwit-config/src/index_config/mod.rs b/quickwit/quickwit-config/src/index_config/mod.rs index fcef34a9366..0cde19166c9 100644 --- a/quickwit/quickwit-config/src/index_config/mod.rs +++ b/quickwit/quickwit-config/src/index_config/mod.rs @@ -273,7 +273,12 @@ impl IndexConfig { hasher.finish() } - /// Compare IndexConfig config level fingerprints + /// Compare IndexConfig level fingerprints + /// + /// This method is meant to enable IndexConfig level fingerprint comparison + /// without taking the risk of mixing them up with pipeline level + /// fingerprints (computed by + /// [`crate::indexing_pipeline_params_fingerprint()`]). pub fn equals_fingerprint(&self, other: &Self) -> bool { self.indexing_params_fingerprint() == other.indexing_params_fingerprint() } diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index 20f663dc102..8a157190c71 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -288,7 +288,8 @@ pub trait TestableForRegression: Serialize + DeserializeOwned { fn assert_equality(&self, other: &Self); } -/// Return a fingerprint of all parameters that should trigger an indexing pipeline restart. +/// Returns a fingerprint (a hash) of all the parameters that should force an +/// indexing pipeline to restart upon index or source config updates. pub fn indexing_pipeline_params_fingerprint( index_config: &IndexConfig, source_config: &SourceConfig, diff --git a/quickwit/quickwit-config/src/source_config/mod.rs b/quickwit/quickwit-config/src/source_config/mod.rs index c0b9eab809f..68ac6247d0e 100644 --- a/quickwit/quickwit-config/src/source_config/mod.rs +++ b/quickwit/quickwit-config/src/source_config/mod.rs @@ -24,6 +24,7 @@ use std::hash::{Hash, Hasher}; use std::num::NonZeroUsize; use std::str::FromStr; +use anyhow::ensure; use bytes::Bytes; use quickwit_common::is_false; use quickwit_common::uri::Uri; @@ -76,19 +77,7 @@ pub struct SourceConfig { impl SourceConfig { pub fn source_type(&self) -> SourceType { - match self.source_params { - SourceParams::File(_) => SourceType::File, - SourceParams::Ingest => SourceType::IngestV2, - SourceParams::IngestApi => SourceType::IngestV1, - SourceParams::IngestCli => SourceType::Cli, - SourceParams::Kafka(_) => SourceType::Kafka, - SourceParams::Kinesis(_) => SourceType::Kinesis, - SourceParams::PubSub(_) => SourceType::PubSub, - SourceParams::Pulsar(_) => SourceType::Pulsar, - SourceParams::Stdin => SourceType::Stdin, - SourceParams::Vec(_) => SourceType::Vec, - SourceParams::Void(_) => SourceType::Void, - } + self.source_params.source_type() } // TODO: Remove after source factory refactor. @@ -145,7 +134,7 @@ impl SourceConfig { } } - /// Return a fingerprint of parameters relevant for indexers + /// Returns a fingerprint of parameters relevant for indexers. /// /// This should remain private to this crate to avoid confusion with the /// full indexing pipeline fingerprint that also includes the index config's @@ -268,6 +257,51 @@ impl SourceParams { pub fn void() -> Self { Self::Void(VoidSourceParams) } + + fn source_type(&self) -> SourceType { + match self { + SourceParams::File(_) => SourceType::File, + SourceParams::Ingest => SourceType::IngestV2, + SourceParams::IngestApi => SourceType::IngestV1, + SourceParams::IngestCli => SourceType::Cli, + SourceParams::Kafka(_) => SourceType::Kafka, + SourceParams::Kinesis(_) => SourceType::Kinesis, + SourceParams::PubSub(_) => SourceType::PubSub, + SourceParams::Pulsar(_) => SourceType::Pulsar, + SourceParams::Stdin => SourceType::Stdin, + SourceParams::Vec(_) => SourceType::Vec, + SourceParams::Void(_) => SourceType::Void, + } + } + + fn validate_update(&self, new_source_params: &SourceParams) -> anyhow::Result<()> { + match (self, new_source_params) { + ( + SourceParams::File(FileSourceParams::Notifications(current)), + SourceParams::File(FileSourceParams::Notifications(new)), + ) => current.validate_update(new), + (SourceParams::Kafka(current), SourceParams::Kafka(new)) => { + current.validate_update(new) + } + (SourceParams::Kinesis(current), SourceParams::Kinesis(new)) => { + current.validate_update(new) + } + (SourceParams::PubSub(current), SourceParams::PubSub(new)) => { + current.validate_update(new) + } + (SourceParams::Pulsar(current), SourceParams::Pulsar(new)) => { + current.validate_update(new) + } + (current, new) if current.source_type() != new.source_type() => Err(anyhow::anyhow!( + "source type cannot be changed, current type {}", + current.source_type(), + )), + _ => Err(anyhow::anyhow!( + "source type {} cannot be updated", + self.source_type(), + )), + } + } } #[derive(Clone, Copy, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] @@ -309,6 +343,17 @@ pub enum FileSourceNotification { Sqs(FileSourceSqs), } +impl FileSourceNotification { + fn validate_update(&self, other: &Self) -> anyhow::Result<()> { + match (self, other) { + (Self::Sqs(_), Self::Sqs(_)) => { + // changing the queue or the deduplication settings should be fine + Ok(()) + } + } + } +} + #[derive(Clone, Debug, Eq, PartialEq, Deserialize, Serialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub(super) struct FileSourceParamsForSerde { @@ -397,6 +442,16 @@ pub struct KafkaSourceParams { pub enable_backfill_mode: bool, } +impl KafkaSourceParams { + fn validate_update(&self, other: &Self) -> anyhow::Result<()> { + // Updating the topic would likely mess up the checkpoints because the + // Kafka partition IDs are used as metastore checkpoint PartitionId + // and there uniqueness is not guaranteed across topics. + ensure!(self.topic == other.topic, "Kafka topic cannot be updated"); + Ok(()) + } +} + #[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(deny_unknown_fields)] pub struct PubSubSourceParams { @@ -419,6 +474,13 @@ pub struct PubSubSourceParams { pub max_messages_per_pull: Option, } +impl PubSubSourceParams { + fn validate_update(&self, _other: &Self) -> anyhow::Result<()> { + // experimental source, no validation is performed + Ok(()) + } +} + #[derive(Clone, Debug, Eq, PartialEq, Hash, Serialize, Deserialize, utoipa::ToSchema)] #[serde(rename_all = "lowercase")] pub enum RegionOrEndpoint { @@ -437,6 +499,23 @@ pub struct KinesisSourceParams { pub enable_backfill_mode: bool, } +impl KinesisSourceParams { + fn validate_update(&self, other: &Self) -> anyhow::Result<()> { + // Changing the stream would likely mess up the checkpoints because the + // Kinesis shard IDs are used as metastore checkpoint PartitionId, and + // there uniqueness is only guarantied within a stream. + ensure!( + self.stream_name == other.stream_name, + "Kinesis stream_name cannot be updated" + ); + ensure!( + self.region_or_endpoint == other.region_or_endpoint, + "Kinesis region or endpoint cannot be updated" + ); + Ok(()) + } +} + #[derive(Clone, Debug, Eq, PartialEq, Deserialize)] #[serde(deny_unknown_fields)] struct KinesisSourceParamsInner { @@ -503,6 +582,15 @@ pub struct PulsarSourceParams { pub authentication: Option, } +impl PulsarSourceParams { + fn validate_update(&self, _other: &Self) -> anyhow::Result<()> { + // In the Pulsar source, we use use combinations of the topic+partition + // (generated by the Pulsar client library) as metastore checkpoint + // PartitionId, and those are guarantied to be unique across topics. + Ok(()) + } +} + #[derive( Clone, Debug, Eq, PartialEq, Hash, serde::Serialize, serde::Deserialize, utoipa::ToSchema, )] @@ -1400,39 +1488,56 @@ mod tests { let file_content = std::fs::read(&source_config_filepath).unwrap(); let source_config_uri = Uri::from_str(&source_config_filepath).unwrap(); let config_format = ConfigFormat::sniff_from_uri(&source_config_uri).unwrap(); - let mut existing_source_config = - load_source_config_from_user_config(config_format, &file_content).unwrap(); - existing_source_config.num_pipelines = NonZero::new(4).unwrap(); - let new_source_config = - load_source_config_update(config_format, &file_content, &existing_source_config) - .unwrap(); + { + let mut existing_source_config = + load_source_config_from_user_config(config_format, &file_content).unwrap(); + existing_source_config.num_pipelines = NonZero::new(4).unwrap(); + let new_source_config = + load_source_config_update(config_format, &file_content, &existing_source_config) + .unwrap(); - let expected_source_config = SourceConfig { - source_id: "hdfs-logs-kafka-source".to_string(), - num_pipelines: NonZeroUsize::new(2).unwrap(), - enabled: true, - source_params: SourceParams::Kafka(KafkaSourceParams { - topic: "cloudera-cluster-logs".to_string(), - client_log_level: None, - client_params: json! {{"bootstrap.servers": "localhost:9092"}}, + let expected_source_config = SourceConfig { + source_id: "hdfs-logs-kafka-source".to_string(), + num_pipelines: NonZeroUsize::new(2).unwrap(), + enabled: true, + source_params: SourceParams::Kafka(KafkaSourceParams { + topic: "cloudera-cluster-logs".to_string(), + client_log_level: None, + client_params: json! {{"bootstrap.servers": "localhost:9092"}}, + enable_backfill_mode: false, + }), + transform_config: Some(TransformConfig { + vrl_script: ".message = downcase(string!(.message))".to_string(), + timezone: "local".to_string(), + }), + input_format: SourceInputFormat::Json, + }; + assert_eq!(new_source_config, expected_source_config); + assert_eq!(new_source_config.num_pipelines.get(), 2); + } + { + // the source type cannot be updated + let mut existing_source_config = + load_source_config_from_user_config(config_format, &file_content).unwrap(); + existing_source_config.source_params = SourceParams::Kinesis(KinesisSourceParams { + stream_name: "my-stream".to_string(), + region_or_endpoint: None, enable_backfill_mode: false, - }), - transform_config: Some(TransformConfig { - vrl_script: ".message = downcase(string!(.message))".to_string(), - timezone: "local".to_string(), - }), - input_format: SourceInputFormat::Json, - }; - assert_eq!(new_source_config, expected_source_config); - assert_eq!(new_source_config.num_pipelines.get(), 2); - - // the source type cannot be updated - existing_source_config.source_params = SourceParams::Kinesis(KinesisSourceParams { - stream_name: "my-stream".to_string(), - region_or_endpoint: None, - enable_backfill_mode: false, - }); - load_source_config_update(config_format, &file_content, &existing_source_config) - .unwrap_err(); + }); + load_source_config_update(config_format, &file_content, &existing_source_config) + .unwrap_err(); + } + { + // the topic cannot be updated + let mut existing_source_config = + load_source_config_from_user_config(config_format, &file_content).unwrap(); + let SourceParams::Kafka(kafka_params) = &mut existing_source_config.source_params + else { + panic!("expected Kafka source params"); + }; + kafka_params.topic = "other_topic_name".to_string(); + load_source_config_update(config_format, &file_content, &existing_source_config) + .unwrap_err(); + } } } diff --git a/quickwit/quickwit-config/src/source_config/serialize.rs b/quickwit/quickwit-config/src/source_config/serialize.rs index e3c3d03f9d1..54287f4aa6f 100644 --- a/quickwit/quickwit-config/src/source_config/serialize.rs +++ b/quickwit/quickwit-config/src/source_config/serialize.rs @@ -64,6 +64,11 @@ pub fn load_source_config_from_user_config( source_config_for_serialization.validate_and_build() } +/// Parses and validates a [`SourceConfig`] update. +/// +/// Ensures that the new configuration is valid in itself and compared to the +/// current source config. If the new configuration omits some fields, the +/// default values will be used, not those of the current source config. pub fn load_source_config_update( config_format: ConfigFormat, config_content: &[u8], @@ -81,11 +86,9 @@ pub fn load_source_config_update( new_source_config.source_id ); - ensure!( - current_source_config.source_type() == new_source_config.source_type(), - "source type cannot be updated, current type: {}", - current_source_config.source_type(), - ); + current_source_config + .source_params + .validate_update(&new_source_config.source_params)?; Ok(new_source_config) } diff --git a/quickwit/quickwit-index-management/src/index.rs b/quickwit/quickwit-index-management/src/index.rs index 401cf10dbfa..31d5aa704c2 100644 --- a/quickwit/quickwit-index-management/src/index.rs +++ b/quickwit/quickwit-index-management/src/index.rs @@ -495,8 +495,8 @@ impl IndexService { UpdateSourceRequest::try_from_source_config(index_uid.clone(), &source_config)?; self.metastore.update_source(update_source_request).await?; info!( - "source `{}` successfully updated for index `{}`", - source_id, index_uid.index_id, + "source `{source_id}` successfully updated for index `{}`", + index_uid.index_id ); let index_metadata_request = IndexMetadataRequest::for_index_id(index_uid.index_id); let source = self diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index bf7d40afa2e..386d41e0d7c 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -327,12 +327,19 @@ impl IndexingService { let params_fingerprint = indexing_pipeline_params_fingerprint(&index_config, &source_config); if let Some(expected_params_fingerprint) = expected_params_fingerprint { + // If the fingerprint of the config freshly fetched from the + // metastore is different from that received from the control plane, + // it means that the config changed again since the last indexing + // plan was built. In this case, postpone the pipeline creation. if params_fingerprint != expected_params_fingerprint { - warn!( - expected_fingerprint = expected_params_fingerprint, - actual_fingerprint = params_fingerprint, - "params fingerprint mismatch" + info!( + index_id = indexing_pipeline_id.index_uid.index_id, + source_id = indexing_pipeline_id.source_id, + expected = expected_params_fingerprint, + actual = params_fingerprint, + "pipeline fingerprint mismatch, postponing pipeline creation" ); + return Ok(()); } } let pipeline_params = IndexingPipelineParams { @@ -1259,8 +1266,13 @@ mod tests { cluster.clone(), ) .await; + let metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id(index_id.clone())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); - // Test `apply plan`. let source_config_1 = SourceConfig { source_id: "test-indexing-service--source-1".to_string(), num_pipelines: NonZeroUsize::MIN, @@ -1269,43 +1281,41 @@ mod tests { transform_config: None, input_format: SourceInputFormat::Json, }; - let add_source_request = - AddSourceRequest::try_from_source_config(index_uid.clone(), &source_config_1).unwrap(); - metastore.add_source(add_source_request).await.unwrap(); - let metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.clone())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - let indexing_tasks = vec![ - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: "test-indexing-service--source-1".to_string(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(0u128)), - params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, - }, - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: "test-indexing-service--source-1".to_string(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(1u128)), - params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, - }, - ]; - indexing_service - .ask_for_res(ApplyIndexingPlanRequest { indexing_tasks }) - .await - .unwrap(); - assert_eq!( - indexing_service_handle - .observe() + { + // Assign 2 indexing tasks + // -> total: 1 source * 2 pipelines + let add_source_request = + AddSourceRequest::try_from_source_config(index_uid.clone(), &source_config_1) + .unwrap(); + metastore.add_source(add_source_request).await.unwrap(); + let indexing_tasks = vec![ + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_1.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(0u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, + }, + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_1.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(1u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, + }, + ]; + indexing_service + .ask_for_res(ApplyIndexingPlanRequest { indexing_tasks }) .await - .num_running_pipelines, - 2 - ); - + .unwrap(); + assert_eq!( + indexing_service_handle + .observe() + .await + .num_running_pipelines, + 2 + ); + } let kafka_params = KafkaSourceParams { topic: "my-topic".to_string(), client_log_level: None, @@ -1320,126 +1330,170 @@ mod tests { transform_config: None, input_format: SourceInputFormat::Json, }; - let add_source_request_2 = - AddSourceRequest::try_from_source_config(index_uid.clone(), &source_config_2).unwrap(); - metastore.add_source(add_source_request_2).await.unwrap(); - - let indexing_tasks = vec![ - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: INGEST_API_SOURCE_ID.to_string(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(3u128)), - params_fingerprint: PARAMS_FINGERPRINT_INGEST_API, - }, - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: "test-indexing-service--source-1".to_string(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(1u128)), - params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, - }, - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: "test-indexing-service--source-1".to_string(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(2u128)), - params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, - }, - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: source_config_2.source_id.clone(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(4u128)), - params_fingerprint: PARAMS_FINGERPRINT_SOURCE_2, - }, - ]; - indexing_service - .ask_for_res(ApplyIndexingPlanRequest { - indexing_tasks: indexing_tasks.clone(), - }) - .await - .unwrap(); - assert_eq!( - indexing_service_handle - .observe() + { + // Assign 2 more indexing tasks (1 new source + activate ingest API source) + // -> total: 2 source * 1 pipeline + 1 source * 2 pipelines + let add_source_request_2 = + AddSourceRequest::try_from_source_config(index_uid.clone(), &source_config_2) + .unwrap(); + metastore.add_source(add_source_request_2).await.unwrap(); + + let indexing_tasks = vec![ + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: INGEST_API_SOURCE_ID.to_string(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(3u128)), + params_fingerprint: PARAMS_FINGERPRINT_INGEST_API, + }, + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_1.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(1u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, + }, + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_1.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(2u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, + }, + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_2.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(4u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_2, + }, + ]; + indexing_service + .ask_for_res(ApplyIndexingPlanRequest { + indexing_tasks: indexing_tasks.clone(), + }) .await - .num_running_pipelines, - 4 - ); - - cluster - .wait_for_ready_members( - |members| { - members - .iter() - .any(|member| member.indexing_tasks.len() == indexing_tasks.len()) + .unwrap(); + assert_eq!( + indexing_service_handle + .observe() + .await + .num_running_pipelines, + 4 + ); + cluster + .wait_for_ready_members( + |members| { + members + .iter() + .any(|member| member.indexing_tasks.len() == indexing_tasks.len()) + }, + Duration::from_secs(5), + ) + .await + .unwrap(); + let self_member = &cluster.ready_members().await[0]; + assert_eq!( + HashSet::<_>::from_iter(self_member.indexing_tasks.iter()), + HashSet::from_iter(indexing_tasks.iter()) + ); + } + { + // Remove 1 task (source_1 runs only 1 pipeline) + // -> total = 3 sources x 1 pipeline each + let indexing_tasks = vec![ + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: INGEST_API_SOURCE_ID.to_string(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(3u128)), + params_fingerprint: PARAMS_FINGERPRINT_INGEST_API, }, - Duration::from_secs(5), - ) - .await - .unwrap(); - - let self_member = &cluster.ready_members().await[0]; - - assert_eq!( - HashSet::<_>::from_iter(self_member.indexing_tasks.iter()), - HashSet::from_iter(indexing_tasks.iter()) - ); - let indexing_tasks = vec![ - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: INGEST_API_SOURCE_ID.to_string(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(3u128)), - params_fingerprint: PARAMS_FINGERPRINT_INGEST_API, - }, - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: "test-indexing-service--source-1".to_string(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(1u128)), - params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, - }, - IndexingTask { - index_uid: Some(metadata.index_uid.clone()), - source_id: source_config_2.source_id.clone(), - shard_ids: Vec::new(), - pipeline_uid: Some(PipelineUid::for_test(4u128)), - params_fingerprint: PARAMS_FINGERPRINT_SOURCE_2, - }, - ]; - indexing_service - .ask_for_res(ApplyIndexingPlanRequest { - indexing_tasks: indexing_tasks.clone(), - }) - .await - .unwrap(); - let indexing_service_obs = indexing_service_handle.observe().await; - assert_eq!(indexing_service_obs.num_running_pipelines, 3); - assert_eq!(indexing_service_obs.num_deleted_queues, 0); - assert_eq!(indexing_service_obs.num_delete_queue_failures, 0); - - indexing_service_handle.process_pending_and_observe().await; - - cluster - .wait_for_ready_members( - |members| { - members - .iter() - .any(|member| member.indexing_tasks.len() == indexing_tasks.len()) + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_1.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(1u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_1, }, - Duration::from_secs(5), - ) - .await - .unwrap(); + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_2.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(4u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_2, + }, + ]; + indexing_service + .ask_for_res(ApplyIndexingPlanRequest { + indexing_tasks: indexing_tasks.clone(), + }) + .await + .unwrap(); + let indexing_service_obs = indexing_service_handle.observe().await; + assert_eq!(indexing_service_obs.num_running_pipelines, 3); + assert_eq!(indexing_service_obs.num_deleted_queues, 0); + assert_eq!(indexing_service_obs.num_delete_queue_failures, 0); + + indexing_service_handle.process_pending_and_observe().await; + + cluster + .wait_for_ready_members( + |members| { + members + .iter() + .any(|member| member.indexing_tasks.len() == indexing_tasks.len()) + }, + Duration::from_secs(5), + ) + .await + .unwrap(); - let self_member = &cluster.ready_members().await[0]; + let self_member = &cluster.ready_members().await[0]; - assert_eq!( - HashSet::<_>::from_iter(self_member.indexing_tasks.iter()), - HashSet::from_iter(indexing_tasks.iter()) - ); + assert_eq!( + HashSet::<_>::from_iter(self_member.indexing_tasks.iter()), + HashSet::from_iter(indexing_tasks.iter()) + ); + } + { + // Rescheduling a task (source_1) with an unexpected fingerprint + // removes the existing pipeline but doesn't start a new one. + // -> total: 2 sources x 1 pipeline + let indexing_tasks = vec![ + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: INGEST_API_SOURCE_ID.to_string(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(3u128)), + params_fingerprint: PARAMS_FINGERPRINT_INGEST_API, + }, + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_1.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(7u128)), + params_fingerprint: 42, + }, + IndexingTask { + index_uid: Some(metadata.index_uid.clone()), + source_id: source_config_2.source_id.clone(), + shard_ids: Vec::new(), + pipeline_uid: Some(PipelineUid::for_test(4u128)), + params_fingerprint: PARAMS_FINGERPRINT_SOURCE_2, + }, + ]; + indexing_service + .ask_for_res(ApplyIndexingPlanRequest { + indexing_tasks: indexing_tasks.clone(), + }) + .await + .unwrap(); + let indexing_service_obs = indexing_service_handle.observe().await; + assert_eq!(indexing_service_obs.num_running_pipelines, 2); + assert_eq!(indexing_service_obs.num_deleted_queues, 0); + assert_eq!(indexing_service_obs.num_delete_queue_failures, 0); + } // Delete index and apply empty plan metastore From 14e6abc6b7adff6ab3d4e8bddb4dd8bddd2c31a7 Mon Sep 17 00:00:00 2001 From: Remi Dettai Date: Thu, 23 Jan 2025 10:43:30 +0100 Subject: [PATCH 11/11] Fix rest handler update test --- quickwit/Cargo.lock | 1 + quickwit/quickwit-indexing/Cargo.toml | 4 +- .../src/source/queue_sources/sqs_queue.rs | 28 +++- quickwit/quickwit-serve/Cargo.toml | 4 + .../src/index_api/rest_handler.rs | 121 ++++++++++-------- 5 files changed, 104 insertions(+), 54 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 2ccb875c421..238ad68fad0 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -6691,6 +6691,7 @@ dependencies = [ "ulid", "utoipa", "vrl", + "warp", ] [[package]] diff --git a/quickwit/quickwit-indexing/Cargo.toml b/quickwit/quickwit-indexing/Cargo.toml index 381bb1151e4..ae9b6c2b0a9 100644 --- a/quickwit/quickwit-indexing/Cargo.toml +++ b/quickwit/quickwit-indexing/Cargo.toml @@ -47,6 +47,7 @@ tracing = { workspace = true } ulid = { workspace = true } utoipa = { workspace = true } vrl = { workspace = true, optional = true } +warp = { workspace = true, optional = true } quickwit-actors = { workspace = true } quickwit-aws = { workspace = true } @@ -84,7 +85,8 @@ sqs = [ "queue-sources", "quickwit-aws/sqs", ] -sqs-localstack-tests = [] +sqs-test-helpers = ["warp"] +sqs-localstack-tests = ["sqs-test-helpers"] vendored-kafka = [ "kafka", "libz-sys/static", diff --git a/quickwit/quickwit-indexing/src/source/queue_sources/sqs_queue.rs b/quickwit/quickwit-indexing/src/source/queue_sources/sqs_queue.rs index 49d5923a9e1..25e7dfe1e95 100644 --- a/quickwit/quickwit-indexing/src/source/queue_sources/sqs_queue.rs +++ b/quickwit/quickwit-indexing/src/source/queue_sources/sqs_queue.rs @@ -261,10 +261,11 @@ pub(crate) async fn check_connectivity(queue_url: &str) -> anyhow::Result<()> { Ok(()) } -#[cfg(feature = "sqs-localstack-tests")] +#[cfg(feature = "sqs-test-helpers")] pub mod test_helpers { use aws_sdk_sqs::types::QueueAttributeName; use ulid::Ulid; + use warp::Filter; use super::*; @@ -316,6 +317,31 @@ pub mod test_helpers { .unwrap() .to_string() } + + /// Runs a mock SQS GetQueueAttributes endpoint to enable creating SQS + /// sources that pass the connectivity check + /// + /// Rerturns the queue URL to use for the source and a guard for the + /// temporary mock server + pub fn start_mock_sqs_get_queue_attributes_endpoint() -> (String, oneshot::Sender<()>) { + let hello = warp::path!().map(|| "{}"); + let (tx, rx) = oneshot::channel(); + let (addr, server) = + warp::serve(hello).bind_with_graceful_shutdown(([127, 0, 0, 1], 0), async { + rx.await.ok(); + }); + tokio::spawn(server); + let queue_url = format!("http://{}:{}/", addr.ip(), addr.port()); + (queue_url, tx) + } + + #[tokio::test] + async fn test_mock_sqs_get_queue_attributes_endpoint() { + let (queue_url, _shutdown) = start_mock_sqs_get_queue_attributes_endpoint(); + check_connectivity(&queue_url).await.unwrap(); + drop(_shutdown); + check_connectivity(&queue_url).await.unwrap_err(); + } } #[cfg(test)] diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index b82db775761..4b030247a74 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -98,3 +98,7 @@ pprof = [ "dep:pprof" ] testsuite = [] +sqs-for-tests = [ + "quickwit-indexing/sqs", + "quickwit-indexing/sqs-test-helpers" +] diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index d1adb91a3a6..41ef0df194a 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -1111,9 +1111,13 @@ mod tests { } } + #[cfg(feature = "sqs-for-tests")] #[tokio::test] async fn test_update_source() { + use quickwit_indexing::source::sqs_queue::test_helpers::start_mock_sqs_get_queue_attributes_endpoint; + let metastore = metastore_for_test(); + let (queue_url, _guard) = start_mock_sqs_get_queue_attributes_endpoint(); let index_service = IndexService::new(metastore.clone(), StorageResolver::unconfigured()); let mut node_config = NodeConfig::for_test(); node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); @@ -1137,64 +1141,77 @@ mod tests { assert_json_include!(actual: resp_json, expected: expected_response_json); // Create source. - let source_config_body = r#"{"version": "0.7", "source_id": "vec-source", "source_type": "vec", "params": {"docs": [], "batch_num_docs": 10}}"#; + let source_config_body = serde_json::json!({ + "version": "0.7", + "source_id": "sqs-source", + "source_type": "file", + "params": {"notifications": [{"type": "sqs", "queue_url": queue_url, "message_type": "s3_notification"}]}, + }); let resp = warp::test::request() .path("/indexes/hdfs-logs/sources") .method("POST") - .json(&true) - .body(source_config_body) + .json(&source_config_body) .reply(&index_management_handler) .await; - assert_eq!(resp.status(), 200); - - // Update the source. - let update_source_config_body = r#"{"version": "0.7", "source_id": "vec-source", "source_type": "vec", "params": {"docs": [], "batch_num_docs": 20}}"#; - let resp = warp::test::request() - .path("/indexes/hdfs-logs/sources/vec-source") - .method("PUT") - .json(&true) - .body(update_source_config_body) - .reply(&index_management_handler) - .await; - assert_eq!(resp.status(), 200); - // Check that the source has been updated. - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - assert!(index_metadata.sources.contains_key("vec-source")); - let source_config = index_metadata.sources.get("vec-source").unwrap(); - assert_eq!(source_config.source_type(), SourceType::Vec); - assert_eq!( - source_config.source_params, - SourceParams::Vec(VecSourceParams { - docs: Vec::new(), - batch_num_docs: 20, - partition: "".to_string(), - }) - ); + let resp_body = std::str::from_utf8(resp.body()).unwrap(); + assert_eq!(resp.status(), 200, "{resp_body}"); - // Update the source with a different source_id (forbidden) - let update_source_config_body = r#"{"version": "0.7", "source_id": "other-source-id", "source_type": "vec", "params": {"docs": [], "batch_num_docs": 20}}"#; - let resp = warp::test::request() - .path("/indexes/hdfs-logs/sources/vec-source") - .method("PUT") - .json(&true) - .body(update_source_config_body) - .reply(&index_management_handler) - .await; - assert_eq!(resp.status(), 400); - // Check that the source hasn't been updated. - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - assert!(index_metadata.sources.contains_key("vec-source")); - assert!(!index_metadata.sources.contains_key("other-source-id")); + { + // Update the source. + let update_source_config_body = serde_json::json!({ + "version": "0.7", + "source_id": "sqs-source", + "source_type": "file", + "params": {"notifications": [{"type": "sqs", "queue_url": queue_url, "message_type": "s3_notification"}]}, + }); + let resp = warp::test::request() + .path("/indexes/hdfs-logs/sources/sqs-source") + .method("PUT") + .json(&update_source_config_body) + .reply(&index_management_handler) + .await; + let resp_body = std::str::from_utf8(resp.body()).unwrap(); + assert_eq!(resp.status(), 200, "{resp_body}"); + // Check that the source has been updated. + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + let metastore_source_config = index_metadata.sources.get("sqs-source").unwrap(); + assert_eq!(metastore_source_config.source_type(), SourceType::File); + assert_eq!( + metastore_source_config, + &serde_json::from_value(update_source_config_body).unwrap(), + ); + } + { + // Update the source with a different source_id (forbidden) + let update_source_config_body = serde_json::json!({ + "version": "0.7", + "source_id": "new-source-id", + "source_type": "file", + "params": {"notifications": [{"type": "sqs", "queue_url": queue_url, "message_type": "s3_notification"}]}, + }); + let resp = warp::test::request() + .path("/indexes/hdfs-logs/sources/sqs-source") + .method("PUT") + .json(&update_source_config_body) + .reply(&index_management_handler) + .await; + let resp_body = std::str::from_utf8(resp.body()).unwrap(); + assert_eq!(resp.status(), 400, "{resp_body}"); + // Check that the source hasn't been updated. + let index_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id("hdfs-logs".to_string())) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert!(index_metadata.sources.contains_key("sqs-source")); + assert!(!index_metadata.sources.contains_key("other-source-id")); + } } #[tokio::test]