From 6bd49311fcf59a075a917beac06ae32b5b0fa8bc Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Tue, 27 Jan 2026 22:47:06 +0100 Subject: [PATCH 01/22] first stab --- quickwit/Cargo.lock | 102 ++++++++++++ quickwit/Cargo.toml | 5 + quickwit/quickwit-config/src/lib.rs | 4 +- .../quickwit-config/src/node_config/mod.rs | 53 ++++++ quickwit/quickwit-lambda/Cargo.toml | 49 ++++++ .../quickwit-lambda/src/bin/leaf_search.rs | 45 +++++ quickwit/quickwit-lambda/src/config.rs | 65 ++++++++ quickwit/quickwit-lambda/src/context.rs | 78 +++++++++ quickwit/quickwit-lambda/src/error.rs | 98 +++++++++++ quickwit/quickwit-lambda/src/handler.rs | 96 +++++++++++ .../quickwit-lambda/src/invoker/aws_lambda.rs | 154 +++++++++++++++++ quickwit/quickwit-lambda/src/invoker/mod.rs | 43 +++++ quickwit/quickwit-lambda/src/lib.rs | 32 ++++ .../protos/quickwit/search.proto | 13 ++ quickwit/quickwit-search/src/lib.rs | 2 +- quickwit/rest-api-tests/config/quickwit.yaml | 157 ++++++++++++++++++ quickwit/rest-api-tests/quickwit | 1 + 17 files changed, 994 insertions(+), 3 deletions(-) create mode 100644 quickwit/quickwit-lambda/Cargo.toml create mode 100644 quickwit/quickwit-lambda/src/bin/leaf_search.rs create mode 100644 quickwit/quickwit-lambda/src/config.rs create mode 100644 quickwit/quickwit-lambda/src/context.rs create mode 100644 quickwit/quickwit-lambda/src/error.rs create mode 100644 quickwit/quickwit-lambda/src/handler.rs create mode 100644 quickwit/quickwit-lambda/src/invoker/aws_lambda.rs create mode 100644 quickwit/quickwit-lambda/src/invoker/mod.rs create mode 100644 quickwit/quickwit-lambda/src/lib.rs create mode 100644 quickwit/rest-api-tests/config/quickwit.yaml create mode 120000 quickwit/rest-api-tests/quickwit diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 1e1c0482df0..2a6628dbc1c 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -532,6 +532,29 @@ dependencies = [ "tracing", ] +[[package]] +name = "aws-sdk-lambda" +version = "1.112.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1cad866b2e51c3af758e5c8bb941a8c904262663fde53019aa1c5093172c54d3" +dependencies = [ + "aws-credential-types", + "aws-runtime", + "aws-smithy-async", + "aws-smithy-eventstream", + "aws-smithy-http 0.62.6", + "aws-smithy-json 0.61.9", + "aws-smithy-runtime", + "aws-smithy-runtime-api", + "aws-smithy-types", + "aws-types", + "bytes", + "fastrand 2.3.0", + "http 0.2.12", + "regex-lite", + "tracing", +] + [[package]] name = "aws-sdk-s3" version = "1.62.0" @@ -4387,6 +4410,55 @@ dependencies = [ "rustversion", ] +[[package]] +name = "lambda_runtime" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ed49669d6430292aead991e19bf13153135a884f916e68f32997c951af637ebe" +dependencies = [ + "async-stream", + "base64 0.22.1", + "bytes", + "futures", + "http 1.4.0", + "http-body 1.0.1", + "http-body-util", + "http-serde", + "hyper 1.8.1", + "hyper-util", + "lambda_runtime_api_client", + "pin-project", + "serde", + "serde_json", + "serde_path_to_error", + "tokio", + "tokio-stream", + "tower 0.4.13", + "tower-layer", + "tracing", +] + +[[package]] +name = "lambda_runtime_api_client" +version = "0.11.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c90a10f094475a34a04da2be11686c4dcfe214d93413162db9ffdff3d3af293a" +dependencies = [ + "bytes", + "futures-channel", + "futures-util", + "http 1.4.0", + "http-body 1.0.1", + "http-body-util", + "hyper 1.8.1", + "hyper-util", + "tokio", + "tower 0.4.13", + "tower-service", + "tracing", + "tracing-subscriber", +] + [[package]] name = "lazy_static" version = "1.5.0" @@ -7275,6 +7347,36 @@ dependencies = [ "utoipa", ] +[[package]] +name = "quickwit-lambda" +version = "0.8.0" +dependencies = [ + "anyhow", + "async-trait", + "aws-config", + "aws-sdk-lambda", + "base64 0.22.1", + "bytes", + "bytesize", + "lambda_runtime", + "mockall", + "prost 0.14.1", + "quickwit-aws", + "quickwit-common", + "quickwit-config", + "quickwit-doc-mapper", + "quickwit-proto", + "quickwit-search", + "quickwit-storage", + "serde", + "serde_json", + "tempfile", + "thiserror 2.0.17", + "tokio", + "tracing", + "tracing-subscriber", +] + [[package]] name = "quickwit-macros" version = "0.8.0" diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 2eaaef4c023..7aa2445e98b 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -19,6 +19,7 @@ members = [ "quickwit-integration-tests", "quickwit-jaeger", "quickwit-janitor", + "quickwit-lambda", "quickwit-macros", "quickwit-metastore", @@ -56,6 +57,7 @@ default-members = [ "quickwit-integration-tests", "quickwit-jaeger", "quickwit-janitor", + "quickwit-lambda", "quickwit-macros", "quickwit-metastore", "quickwit-opentelemetry", @@ -141,6 +143,7 @@ hyper-util = { version = "0.1", default-features = false, features = [ indexmap = { version = "2.12", features = ["serde"] } indicatif = "0.18" itertools = "0.14" +lambda_runtime = "0.13" json_comments = "0.2" libz-sys = "1.1" # Lindera tokenizer 0.30+ versions (tested up to 0.32.3) are currently broken due to upstream build failures. @@ -315,6 +318,7 @@ aws-credential-types = { version = "1.2", features = ["hardcoded-credentials"] } aws-runtime = "1.5" aws-sdk-kinesis = "1.97" aws-sdk-s3 = "=1.62" +aws-sdk-lambda = "1" aws-sdk-sqs = "1.91" aws-smithy-async = "1.2" aws-smithy-http-client = { version = "1.1", features = ["default-client"] } @@ -355,6 +359,7 @@ quickwit-ingest = { path = "quickwit-ingest" } quickwit-integration-tests = { path = "quickwit-integration-tests" } quickwit-jaeger = { path = "quickwit-jaeger" } quickwit-janitor = { path = "quickwit-janitor" } +quickwit-lambda = { path = "quickwit-lambda" } quickwit-macros = { path = "quickwit-macros" } quickwit-metastore = { path = "quickwit-metastore" } quickwit-opentelemetry = { path = "quickwit-opentelemetry" } diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index df108b844c0..59b019e8f0b 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -74,8 +74,8 @@ pub use crate::metastore_config::{ }; pub use crate::node_config::{ CacheConfig, CachePolicy, DEFAULT_QW_CONFIG_PATH, GrpcConfig, IndexerConfig, IngestApiConfig, - JaegerConfig, KeepAliveConfig, NodeConfig, RestConfig, SearcherConfig, SplitCacheLimits, - StorageTimeoutPolicy, TlsConfig, + JaegerConfig, KeepAliveConfig, LambdaConfig, NodeConfig, RestConfig, SearcherConfig, + SplitCacheLimits, StorageTimeoutPolicy, TlsConfig, }; use crate::source_config::serialize::{SourceConfigV0_7, SourceConfigV0_8, VersionedSourceConfig}; pub use crate::storage_config::{ diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index e8c347eb4a5..bfd27984e70 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -300,6 +300,58 @@ pub struct SearcherConfig { pub storage_timeout_policy: Option, pub warmup_memory_budget: ByteSize, pub warmup_single_split_initial_allocation: ByteSize, + /// Lambda configuration for serverless leaf search execution. + #[serde(default)] + pub lambda: LambdaConfig, +} + +/// Configuration for AWS Lambda leaf search execution. +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(deny_unknown_fields, default)] +pub struct LambdaConfig { + /// Enable Lambda execution mode for leaf search. + #[serde(default)] + pub enabled: bool, + /// AWS Lambda function name or ARN. + #[serde(default)] + pub function_name: Option, + /// Optional function qualifier (alias or version). + #[serde(default)] + pub function_qualifier: Option, + /// Maximum number of splits per Lambda invocation. + #[serde(default = "LambdaConfig::default_max_splits_per_invocation")] + pub max_splits_per_invocation: usize, + /// Timeout for Lambda invocations in seconds. + #[serde(default = "LambdaConfig::default_invocation_timeout_secs")] + pub invocation_timeout_secs: u64, + /// Maximum number of concurrent Lambda invocations. + #[serde(default = "LambdaConfig::default_max_concurrent_invocations")] + pub max_concurrent_invocations: usize, +} + +impl Default for LambdaConfig { + fn default() -> Self { + Self { + enabled: false, + function_name: None, + function_qualifier: None, + max_splits_per_invocation: Self::default_max_splits_per_invocation(), + invocation_timeout_secs: Self::default_invocation_timeout_secs(), + max_concurrent_invocations: Self::default_max_concurrent_invocations(), + } + } +} + +impl LambdaConfig { + fn default_max_splits_per_invocation() -> usize { + 10 + } + fn default_invocation_timeout_secs() -> u64 { + 30 + } + fn default_max_concurrent_invocations() -> usize { + 100 + } } #[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] @@ -435,6 +487,7 @@ impl Default for SearcherConfig { storage_timeout_policy: None, warmup_memory_budget: ByteSize::gb(100), warmup_single_split_initial_allocation: ByteSize::gb(1), + lambda: LambdaConfig::default(), } } } diff --git a/quickwit/quickwit-lambda/Cargo.toml b/quickwit/quickwit-lambda/Cargo.toml new file mode 100644 index 00000000000..eb801dd20ce --- /dev/null +++ b/quickwit/quickwit-lambda/Cargo.toml @@ -0,0 +1,49 @@ +[package] +name = "quickwit-lambda" +description = "AWS Lambda support for Quickwit leaf search" + +version.workspace = true +edition.workspace = true +homepage.workspace = true +documentation.workspace = true +repository.workspace = true +authors.workspace = true +license.workspace = true + +[dependencies] +anyhow = { workspace = true } +async-trait = { workspace = true } +base64 = { workspace = true } +bytes = { workspace = true } +bytesize = { workspace = true } +lambda_runtime = { workspace = true } +prost = { workspace = true } +serde = { workspace = true } +serde_json = { workspace = true } +thiserror = { workspace = true } +tokio = { workspace = true } +tracing = { workspace = true } +tracing-subscriber = { workspace = true, features = ["env-filter", "json"] } + +aws-config = { workspace = true } +aws-sdk-lambda = { workspace = true } + +quickwit-aws = { workspace = true } +quickwit-common = { workspace = true } +quickwit-config = { workspace = true } +quickwit-doc-mapper = { workspace = true } +quickwit-proto = { workspace = true } +quickwit-search = { workspace = true } +quickwit-storage = { workspace = true } + +[dev-dependencies] +mockall = { workspace = true } +tempfile = { workspace = true } + +[[bin]] +name = "quickwit-lambda-leaf-search" +path = "src/bin/leaf_search.rs" + +[features] +default = [] +testsuite = [] diff --git a/quickwit/quickwit-lambda/src/bin/leaf_search.rs b/quickwit/quickwit-lambda/src/bin/leaf_search.rs new file mode 100644 index 00000000000..ef137f33293 --- /dev/null +++ b/quickwit/quickwit-lambda/src/bin/leaf_search.rs @@ -0,0 +1,45 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! AWS Lambda binary entry point for Quickwit leaf search. + +use lambda_runtime::{service_fn, Error, LambdaEvent}; +use quickwit_lambda::{handle_leaf_search, LambdaSearcherContext, LeafSearchPayload}; +use tracing::info; +use tracing_subscriber::EnvFilter; + +#[tokio::main] +async fn main() -> Result<(), Error> { + // Initialize tracing with JSON output for CloudWatch + tracing_subscriber::fmt() + .with_env_filter(EnvFilter::from_default_env()) + .json() + .init(); + + info!("Starting Quickwit Lambda leaf search handler"); + + // Pre-initialize context on cold start + let context = LambdaSearcherContext::get_or_init().await; + + info!("Lambda context initialized, starting handler loop"); + + // Run the Lambda handler + lambda_runtime::run(service_fn(|event: LambdaEvent| async move { + let (payload, _lambda_ctx) = event.into_parts(); + handle_leaf_search(payload, context) + .await + .map_err(|e| lambda_runtime::Error::from(e.to_string())) + })) + .await +} diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda/src/config.rs new file mode 100644 index 00000000000..93fd7d146c1 --- /dev/null +++ b/quickwit/quickwit-lambda/src/config.rs @@ -0,0 +1,65 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use bytesize::ByteSize; + +// Re-export LambdaConfig from quickwit-config +pub use quickwit_config::LambdaConfig; + +/// Configuration for the Lambda handler's SearcherContext. +/// These settings are optimized for Lambda's memory constraints. +#[derive(Clone, Debug)] +pub struct LambdaSearcherConfig { + /// Memory allocated to the Lambda function in MB. + pub memory_mb: usize, + + /// Fast field cache capacity (derived from memory_mb). + pub fast_field_cache_capacity: ByteSize, + + /// Split footer cache capacity. + pub split_footer_cache_capacity: ByteSize, + + /// Maximum concurrent split searches within a single Lambda invocation. + pub max_concurrent_split_searches: usize, + + /// Warmup memory budget. + pub warmup_memory_budget: ByteSize, +} + +impl LambdaSearcherConfig { + /// Create a Lambda-optimized searcher config based on the allocated memory. + pub fn for_memory(memory_mb: usize) -> Self { + // Allocate roughly 1/4 of memory to fast field cache + let fast_field_cache_capacity = ByteSize::mb((memory_mb / 4) as u64); + // Fixed reasonable sizes for other caches + let split_footer_cache_capacity = ByteSize::mb(50); + // Warmup budget is about half of memory + let warmup_memory_budget = ByteSize::mb((memory_mb / 2) as u64); + + Self { + memory_mb, + fast_field_cache_capacity, + split_footer_cache_capacity, + max_concurrent_split_searches: 20, + warmup_memory_budget, + } + } +} + +impl Default for LambdaSearcherConfig { + fn default() -> Self { + // Default to 1024 MB Lambda + Self::for_memory(1024) + } +} diff --git a/quickwit/quickwit-lambda/src/context.rs b/quickwit/quickwit-lambda/src/context.rs new file mode 100644 index 00000000000..2272065a922 --- /dev/null +++ b/quickwit/quickwit-lambda/src/context.rs @@ -0,0 +1,78 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::{Arc, OnceLock}; + +use quickwit_config::SearcherConfig; +use quickwit_search::SearcherContext; +use quickwit_storage::StorageResolver; +use tracing::info; + +use crate::config::LambdaSearcherConfig; +use crate::error::LambdaResult; + +/// Lambda-specific searcher context that caches resources across warm invocations. +pub struct LambdaSearcherContext { + pub searcher_context: Arc, + pub storage_resolver: StorageResolver, +} + +impl LambdaSearcherContext { + pub async fn new(config: LambdaSearcherConfig) -> LambdaResult { + info!( + memory_mb = config.memory_mb, + "Initializing Lambda searcher context" + ); + + let searcher_config = create_searcher_config(&config); + let searcher_context = Arc::new(SearcherContext::new(searcher_config, None)); + let storage_resolver = StorageResolver::configured(&Default::default()); + + Ok(Self { + searcher_context, + storage_resolver, + }) + } + + pub async fn get_or_init() -> &'static Self { + static CONTEXT: OnceLock = OnceLock::new(); + + if let Some(ctx) = CONTEXT.get() { + return ctx; + } + + let config = config_from_env(); + let ctx = Self::new(config) + .await + .expect("Failed to initialize Lambda searcher context"); + + let _ = CONTEXT.set(ctx); + CONTEXT.get().unwrap() + } +} + +fn create_searcher_config(config: &LambdaSearcherConfig) -> SearcherConfig { + let mut searcher_config = SearcherConfig::default(); + searcher_config.max_num_concurrent_split_searches = config.max_concurrent_split_searches; + searcher_config +} + +fn config_from_env() -> LambdaSearcherConfig { + let memory_mb = std::env::var("AWS_LAMBDA_FUNCTION_MEMORY_SIZE") + .ok() + .and_then(|s| s.parse().ok()) + .unwrap_or(1024); + + LambdaSearcherConfig::for_memory(memory_mb) +} diff --git a/quickwit/quickwit-lambda/src/error.rs b/quickwit/quickwit-lambda/src/error.rs new file mode 100644 index 00000000000..a38f7774eb7 --- /dev/null +++ b/quickwit/quickwit-lambda/src/error.rs @@ -0,0 +1,98 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt; + +use quickwit_search::SearchError; + +/// Result type for Lambda operations. +pub type LambdaResult = Result; + +/// Errors that can occur during Lambda operations. +#[derive(Debug)] +pub enum LambdaError { + /// Error during Lambda invocation. + Invocation(String), + /// Error serializing/deserializing protobuf. + Serialization(String), + /// Error from the search operation. + Search(SearchError), + /// Lambda function returned an error. + FunctionError(String), + /// Configuration error. + Configuration(String), + /// Internal error. + Internal(String), +} + +impl fmt::Display for LambdaError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + LambdaError::Invocation(msg) => write!(f, "Lambda invocation error: {}", msg), + LambdaError::Serialization(msg) => write!(f, "Serialization error: {}", msg), + LambdaError::Search(err) => write!(f, "Search error: {}", err), + LambdaError::FunctionError(msg) => write!(f, "Lambda function error: {}", msg), + LambdaError::Configuration(msg) => write!(f, "Configuration error: {}", msg), + LambdaError::Internal(msg) => write!(f, "Internal error: {}", msg), + } + } +} + +impl std::error::Error for LambdaError { + fn source(&self) -> Option<&(dyn std::error::Error + 'static)> { + match self { + LambdaError::Search(err) => Some(err), + _ => None, + } + } +} + +impl From for LambdaError { + fn from(err: SearchError) -> Self { + LambdaError::Search(err) + } +} + +impl From for LambdaError { + fn from(err: prost::DecodeError) -> Self { + LambdaError::Serialization(format!("Protobuf decode error: {}", err)) + } +} + +impl From for LambdaError { + fn from(err: prost::EncodeError) -> Self { + LambdaError::Serialization(format!("Protobuf encode error: {}", err)) + } +} + +impl From for LambdaError { + fn from(err: base64::DecodeError) -> Self { + LambdaError::Serialization(format!("Base64 decode error: {}", err)) + } +} + +impl From for LambdaError { + fn from(err: serde_json::Error) -> Self { + LambdaError::Serialization(format!("JSON error: {}", err)) + } +} + +impl From for SearchError { + fn from(err: LambdaError) -> Self { + match err { + LambdaError::Search(search_err) => search_err, + other => SearchError::Internal(other.to_string()), + } + } +} diff --git a/quickwit/quickwit-lambda/src/handler.rs b/quickwit/quickwit-lambda/src/handler.rs new file mode 100644 index 00000000000..b22f21d7415 --- /dev/null +++ b/quickwit/quickwit-lambda/src/handler.rs @@ -0,0 +1,96 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use base64::prelude::*; +use prost::Message; +use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; +use quickwit_search::leaf::multi_index_leaf_search; +use serde::{Deserialize, Serialize}; +use tracing::{info, instrument}; + +use crate::context::LambdaSearcherContext; +use crate::error::{LambdaError, LambdaResult}; + +/// Payload for leaf search Lambda invocation. +#[derive(Debug, Serialize, Deserialize)] +pub struct LeafSearchPayload { + /// Base64-encoded serialized LeafSearchRequest protobuf. + pub payload: String, +} + +/// Response from leaf search Lambda invocation. +#[derive(Debug, Serialize, Deserialize)] +pub struct LeafSearchResponsePayload { + /// Base64-encoded serialized LeafSearchResponse protobuf. + pub payload: String, +} + +/// Handle a leaf search request in Lambda. +#[instrument(skip(ctx), fields(request_id))] +pub async fn handle_leaf_search( + event: LeafSearchPayload, + ctx: &LambdaSearcherContext, +) -> LambdaResult { + // Decode base64 payload + let request_bytes = BASE64_STANDARD + .decode(&event.payload) + .map_err(|e| LambdaError::Serialization(format!("Base64 decode error: {}", e)))?; + + // Deserialize LeafSearchRequest + let leaf_search_request = LeafSearchRequest::decode(&request_bytes[..])?; + + let num_splits: usize = leaf_search_request + .leaf_requests + .iter() + .map(|r| r.split_offsets.len()) + .sum(); + + info!(num_splits, "Processing leaf search request"); + + // Execute leaf search + let leaf_search_response = multi_index_leaf_search( + ctx.searcher_context.clone(), + leaf_search_request, + &ctx.storage_resolver, + ) + .await + .map_err(LambdaError::from)?; + + info!( + num_hits = leaf_search_response.num_hits, + num_successful_splits = leaf_search_response.num_successful_splits, + "Leaf search completed" + ); + + // Serialize response + let response_bytes = leaf_search_response.encode_to_vec(); + let payload = BASE64_STANDARD.encode(&response_bytes); + + Ok(LeafSearchResponsePayload { payload }) +} + +/// Decode a LeafSearchRequest from base64-encoded protobuf bytes. +#[allow(dead_code)] +pub fn decode_leaf_search_request(payload: &str) -> LambdaResult { + let bytes = BASE64_STANDARD + .decode(payload) + .map_err(|e| LambdaError::Serialization(format!("Base64 decode error: {}", e)))?; + LeafSearchRequest::decode(&bytes[..]).map_err(LambdaError::from) +} + +/// Encode a LeafSearchResponse to base64-encoded protobuf bytes. +#[allow(dead_code)] +pub fn encode_leaf_search_response(response: &LeafSearchResponse) -> String { + BASE64_STANDARD.encode(response.encode_to_vec()) +} diff --git a/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs b/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs new file mode 100644 index 00000000000..a2fb3c10b3d --- /dev/null +++ b/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs @@ -0,0 +1,154 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use async_trait::async_trait; +use aws_sdk_lambda::primitives::Blob; +use aws_sdk_lambda::types::InvocationType; +use aws_sdk_lambda::Client as LambdaClient; +use base64::prelude::*; +use prost::Message; +use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; +use tracing::{debug, instrument}; + +use super::RemoteFunctionInvoker; +use crate::config::LambdaConfig; +use crate::error::{LambdaError, LambdaResult}; +use crate::handler::{LeafSearchPayload, LeafSearchResponsePayload}; + +/// AWS Lambda implementation of RemoteFunctionInvoker. +pub struct AwsLambdaInvoker { + client: LambdaClient, + function_name: String, + qualifier: Option, +} + +impl AwsLambdaInvoker { + /// Create a new AWS Lambda invoker with the given configuration. + pub async fn new(config: &LambdaConfig) -> LambdaResult { + let function_name = config + .function_name + .clone() + .ok_or_else(|| LambdaError::Configuration("Lambda function_name is required".into()))?; + + let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; + let client = LambdaClient::new(&aws_config); + + Ok(Self { + client, + function_name, + qualifier: config.function_qualifier.clone(), + }) + } + + /// Create a new AWS Lambda invoker with a custom client. + pub fn with_client( + client: LambdaClient, + function_name: String, + qualifier: Option, + ) -> Self { + Self { + client, + function_name, + qualifier, + } + } +} + +#[async_trait] +impl RemoteFunctionInvoker for AwsLambdaInvoker { + #[instrument(skip(self, request), fields(function_name = %self.function_name))] + async fn invoke_leaf_search( + &self, + request: LeafSearchRequest, + ) -> LambdaResult { + // Serialize request to protobuf bytes, then base64 encode + let request_bytes = request.encode_to_vec(); + let payload = LeafSearchPayload { + payload: BASE64_STANDARD.encode(&request_bytes), + }; + + let payload_json = serde_json::to_vec(&payload)?; + + debug!( + payload_size = payload_json.len(), + "Invoking Lambda function" + ); + + // Invoke Lambda synchronously (RequestResponse) + let mut invoke_builder = self + .client + .invoke() + .function_name(&self.function_name) + .invocation_type(InvocationType::RequestResponse) + .payload(Blob::new(payload_json)); + + if let Some(qualifier) = &self.qualifier { + invoke_builder = invoke_builder.qualifier(qualifier); + } + + let response = invoke_builder + .send() + .await + .map_err(|e| LambdaError::Invocation(e.to_string()))?; + + // Check for function error + if let Some(error) = response.function_error() { + let error_payload = response + .payload() + .map(|b| String::from_utf8_lossy(b.as_ref()).to_string()) + .unwrap_or_default(); + return Err(LambdaError::FunctionError(format!( + "{}: {}", + error, error_payload + ))); + } + + // Deserialize response + let response_payload = response + .payload() + .ok_or_else(|| LambdaError::Invocation("No response payload".into()))?; + + let lambda_response: LeafSearchResponsePayload = + serde_json::from_slice(response_payload.as_ref())?; + + let response_bytes = BASE64_STANDARD + .decode(&lambda_response.payload) + .map_err(|e| LambdaError::Serialization(format!("Base64 decode error: {}", e)))?; + + let leaf_response = LeafSearchResponse::decode(&response_bytes[..])?; + + debug!( + num_hits = leaf_response.num_hits, + "Lambda invocation completed" + ); + + Ok(leaf_response) + } + + async fn health_check(&self) -> LambdaResult<()> { + // Try to get function configuration to verify it exists + let mut get_function = self.client.get_function().function_name(&self.function_name); + + if let Some(qualifier) = &self.qualifier { + get_function = get_function.qualifier(qualifier); + } + + get_function + .send() + .await + .map_err(|e| LambdaError::Configuration(format!("Lambda health check failed: {}", e)))?; + + Ok(()) + } +} diff --git a/quickwit/quickwit-lambda/src/invoker/mod.rs b/quickwit/quickwit-lambda/src/invoker/mod.rs new file mode 100644 index 00000000000..3cfd6d6f2f3 --- /dev/null +++ b/quickwit/quickwit-lambda/src/invoker/mod.rs @@ -0,0 +1,43 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Remote function invoker abstraction for serverless leaf search execution. + +mod aws_lambda; + +pub use aws_lambda::AwsLambdaInvoker; + +use async_trait::async_trait; +use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; + +use crate::error::LambdaResult; + +/// Trait for invoking remote serverless functions for leaf search. +/// +/// This abstraction allows different cloud providers to be supported +/// (AWS Lambda, GCP Cloud Functions, Azure Functions, etc.) +#[async_trait] +pub trait RemoteFunctionInvoker: Send + Sync + 'static { + /// Invoke the remote function with a LeafSearchRequest. + /// + /// The request is serialized to protobuf, sent to the function, + /// and the response is deserialized from protobuf. + async fn invoke_leaf_search( + &self, + request: LeafSearchRequest, + ) -> LambdaResult; + + /// Check if the invoker is properly configured and available. + async fn health_check(&self) -> LambdaResult<()>; +} diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda/src/lib.rs new file mode 100644 index 00000000000..fc7732c70f0 --- /dev/null +++ b/quickwit/quickwit-lambda/src/lib.rs @@ -0,0 +1,32 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! AWS Lambda support for Quickwit leaf search operations. +//! +//! This crate provides: +//! - A Lambda handler that executes leaf search requests +//! - A `RemoteFunctionInvoker` trait for invoking serverless functions +//! - An AWS Lambda implementation of the invoker + +mod config; +mod context; +mod error; +mod handler; +pub mod invoker; + +pub use config::LambdaConfig; +pub use context::LambdaSearcherContext; +pub use error::{LambdaError, LambdaResult}; +pub use handler::{handle_leaf_search, LeafSearchPayload, LeafSearchResponsePayload}; +pub use invoker::{AwsLambdaInvoker, RemoteFunctionInvoker}; diff --git a/quickwit/quickwit-proto/protos/quickwit/search.proto b/quickwit/quickwit-proto/protos/quickwit/search.proto index 04fa0cedf2e..dd18d741394 100644 --- a/quickwit/quickwit-proto/protos/quickwit/search.proto +++ b/quickwit/quickwit-proto/protos/quickwit/search.proto @@ -182,6 +182,14 @@ message ListFields { } // -- Search ------------------- +// Execution mode for leaf search operations. +enum ExecutionMode { + // Default: use gRPC to searcher nodes (existing behavior). + EXECUTION_MODE_GRPC = 0; + // Execute leaf search via remote serverless functions (e.g., AWS Lambda). + EXECUTION_MODE_REMOTE_FUNCTION = 1; +} + message SearchRequest { // Index ID patterns repeated string index_id_patterns = 1; @@ -247,6 +255,11 @@ message SearchRequest { // When an exact index ID is provided (not a pattern), the query fails only if // that index is not found and this parameter is set to `false`. bool ignore_missing_indexes = 18; + + // Execution mode for leaf search operations. + // When set to EXECUTION_MODE_REMOTE_FUNCTION, leaf search is executed via + // serverless functions (e.g., AWS Lambda) instead of gRPC to searcher nodes. + ExecutionMode execution_mode = 19; } enum CountHits { diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 008556d595f..213c1a6a90e 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -23,7 +23,7 @@ mod collector; mod error; mod fetch_docs; mod find_trace_ids_collector; -mod leaf; +pub mod leaf; mod leaf_cache; mod list_fields; mod list_fields_cache; diff --git a/quickwit/rest-api-tests/config/quickwit.yaml b/quickwit/rest-api-tests/config/quickwit.yaml new file mode 100644 index 00000000000..8845b4cbcf9 --- /dev/null +++ b/quickwit/rest-api-tests/config/quickwit.yaml @@ -0,0 +1,157 @@ +# ============================ Node Configuration ============================== +# +# Website: https://quickwit.io +# Docs: https://quickwit.io/docs/configuration/node-config +# +# Configure AWS credentials: https://quickwit.io/docs/guides/aws-setup#aws-credentials +# +# -------------------------------- General settings -------------------------------- +# +# Config file format version. +# +version: 0.8 +# +# Node ID. Must be unique within a cluster. If not set, a random node ID is generated on each startup. +# +# node_id: node-1 +# +# Quickwit opens three sockets. +# - for its HTTP server, hosting the UI and the REST API (TCP) +# - for its gRPC service (TCP) +# - for its Gossip cluster membership service (UDP) +# +# All three services are bound to the same host and a different port. The host can be an IP address or a hostname. +# +# Default HTTP server host is `127.0.0.1` and default HTTP port is 7280. +# The default host value was chosen to avoid exposing the node to the open-world without users' explicit consent. +# This allows for testing Quickwit in single-node mode or with multiple nodes running on the same host and listening +# on different ports. However, in cluster mode, using this value is never appropriate because it causes the node to +# ignore incoming traffic. +# There are two options to set up a node in cluster mode: +# 1. specify the node's hostname or IP +# 2. pass `0.0.0.0` and let Quickwit do its best to discover the node's IP (see `advertise_address`) +# +# listen_address: 127.0.0.1 +# + + #dd_application_key: some_app_key + + +# rest: +# listen_port: 7280 +# cors_allow_origins: +# - "http://localhost:3000" +# extra_headers: +# x-header-1: header-value-1 +# x-header-2: header-value-2 +# +# grpc: +# max_message_size: 10 MiB +# +# IP address advertised by the node, i.e. the IP address that peer nodes should use to connect to the node for RPCs. +# The environment variable `QW_ADVERTISE_ADDRESS` can also be used to override this value. +# The default advertise address is `listen_address`. If `listen_address` is unspecified (`0.0.0.0`), +# Quickwit attempts to sniff the node's IP by scanning the available network interfaces. +# advertise_address: 192.168.0.42 +# +# In order to join a cluster, one needs to specify a list of +# seeds to connect to. If no port is specified, Quickwit will assume +# the seeds are using the same port as the current node gossip port. +# By default, the peer seed list is empty. +# +# peer_seeds: +# - quickwit-searcher-0.local +# - quickwit-searcher-1.local:10000 +# +# Path to directory where temporary data (caches, intermediate indexing data structures) +# is stored. Defaults to `./qwdata`. +# +# data_dir: /path/to/data/dir +# +# Metastore URI. Defaults to `data_dir/indexes#polling_interval=30s`, +# which is a file-backed metastore and mostly convenient for testing. A cluster would +# require a metastore backed by Amzon S3 or PostgreSQL. +# +# metastore_uri: s3://your-bucket/indexes +# metastore_uri: postgres://username:password@host:port/db +# +# When using a file-backed metastore, the state of the metastore will be cached forever. +# If you are indexing and searching from different processes, it is possible to periodically +# refresh the state of the metastore on the searcher using the `polling_interval` hashtag. +# +# metastore_uri: s3://your-bucket/indexes#polling_interval=30s +# +# Default index root URI, which defines where index data (splits) is stored, +# following the scheme `{default_index_root_uri}/{index-id}`. Defaults to `{data_dir}/indexes`. +# +# default_index_root_uri: s3://your-bucket/indexes +# +# -------------------------------- Storage settings -------------------------------- +# https://quickwit.io/docs/configuration/node-config#storage-configuration +# +# Hardcoding credentials into configuration files is not secure and strongly +# discouraged. Prefer the alternative authentication methods that your storage +# backend may provide. +# +# storage: +# azure: +# account: ${QW_AZURE_STORAGE_ACCOUNT} +# access_key: ${QW_AZURE_STORAGE_ACCESS_KEY} +# +# s3: +# access_key_id: ${AWS_ACCESS_KEY_ID} +# secret_access_key: ${AWS_SECRET_ACCESS_KEY} +# region: ${AWS_REGION} +# endpoint: ${QW_S3_ENDPOINT} +# force_path_style_access: ${QW_S3_FORCE_PATH_STYLE_ACCESS:-false} +# disable_multi_object_delete: false +# disable_multipart_upload: false +# +# -------------------------------- Metastore settings -------------------------------- +# https://quickwit.io/docs/configuration/node-config#metastore-configuration +# +# metastore: +# postgres: +# min_connections: 0 +# max_connections: 10 +# acquire_connection_timeout: 10s +# idle_connection_timeout: 10min +# max_connection_lifetime: 30min +# +# -------------------------------- Indexer settings -------------------------------- +# https://quickwit.io/docs/configuration/node-config#indexer-configuration + +indexer: + enable_otlp_endpoint: ${QW_ENABLE_OTLP_ENDPOINT:-true} +# split_store_max_num_bytes: 100G +# split_store_max_num_splits: 1000 +# max_concurrent_split_uploads: 12 +# +# +# -------------------------------- Ingest API settings ------------------------------ +# https://quickwit.io/docs/configuration/node-config#ingest-api-configuration +# +# ingest_api: +# max_queue_memory_usage: 2GiB +# max_queue_disk_usage: 4GiB +# content_length_limit: 10MiB +# +# -------------------------------- Searcher settings -------------------------------- +# https://quickwit.io/docs/configuration/node-config#searcher-configuration +# +searcher: +# fast_field_cache_capacity: 1G +# split_footer_cache_capacity: 500M + partial_request_cache_capacity: 0 +# max_num_concurrent_split_streams: 100 +# max_num_concurrent_split_searches: 100 +# aggregation_memory_limit: 500M +# aggregation_bucket_limit: 65000 +# split_cache: +# max_num_bytes: 1G +# max_num_splits: 10000 +# num_concurrent_downloads: 1 +# -------------------------------- Jaeger settings -------------------------------- + +jaeger: + enable_endpoint: ${QW_ENABLE_JAEGER_ENDPOINT:-true} diff --git a/quickwit/rest-api-tests/quickwit b/quickwit/rest-api-tests/quickwit new file mode 120000 index 00000000000..d48e118f0f9 --- /dev/null +++ b/quickwit/rest-api-tests/quickwit @@ -0,0 +1 @@ +/Users/paul.masurel/git/quickwit/quickwit/target/debug/quickwit \ No newline at end of file From 0eb6e2837db35b5e9653aad855e0e920bc101522 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Tue, 27 Jan 2026 22:47:29 +0100 Subject: [PATCH 02/22] second stab --- .../quickwit-lambda/src/invoker/aws_lambda.rs | 39 ++---- quickwit/quickwit-lambda/src/invoker/mod.rs | 28 +--- quickwit/quickwit-lambda/src/lib.rs | 9 +- .../src/codegen/quickwit/quickwit.search.rs | 36 ++++++ quickwit/quickwit-search/src/lib.rs | 3 + .../quickwit-search/src/remote_function.rs | 33 +++++ quickwit/quickwit-search/src/root.rs | 122 ++++++++++++++---- .../quickwit-search/src/scroll_context.rs | 1 + quickwit/quickwit-search/src/service.rs | 1 + .../src/elasticsearch_api/rest_handler.rs | 1 + .../src/search_api/rest_handler.rs | 1 + 11 files changed, 192 insertions(+), 82 deletions(-) create mode 100644 quickwit/quickwit-search/src/remote_function.rs diff --git a/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs b/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs index a2fb3c10b3d..1f78b050660 100644 --- a/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs +++ b/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs @@ -19,9 +19,9 @@ use aws_sdk_lambda::Client as LambdaClient; use base64::prelude::*; use prost::Message; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; +use quickwit_search::{RemoteFunctionInvoker, SearchError}; use tracing::{debug, instrument}; -use super::RemoteFunctionInvoker; use crate::config::LambdaConfig; use crate::error::{LambdaError, LambdaResult}; use crate::handler::{LeafSearchPayload, LeafSearchResponsePayload}; @@ -71,14 +71,15 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { async fn invoke_leaf_search( &self, request: LeafSearchRequest, - ) -> LambdaResult { + ) -> Result { // Serialize request to protobuf bytes, then base64 encode let request_bytes = request.encode_to_vec(); let payload = LeafSearchPayload { payload: BASE64_STANDARD.encode(&request_bytes), }; - let payload_json = serde_json::to_vec(&payload)?; + let payload_json = serde_json::to_vec(&payload) + .map_err(|e| SearchError::Internal(format!("JSON serialization error: {}", e)))?; debug!( payload_size = payload_json.len(), @@ -100,7 +101,7 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { let response = invoke_builder .send() .await - .map_err(|e| LambdaError::Invocation(e.to_string()))?; + .map_err(|e| SearchError::Internal(format!("Lambda invocation error: {}", e)))?; // Check for function error if let Some(error) = response.function_error() { @@ -108,8 +109,8 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { .payload() .map(|b| String::from_utf8_lossy(b.as_ref()).to_string()) .unwrap_or_default(); - return Err(LambdaError::FunctionError(format!( - "{}: {}", + return Err(SearchError::Internal(format!( + "Lambda function error: {}: {}", error, error_payload ))); } @@ -117,16 +118,18 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { // Deserialize response let response_payload = response .payload() - .ok_or_else(|| LambdaError::Invocation("No response payload".into()))?; + .ok_or_else(|| SearchError::Internal("No response payload from Lambda".into()))?; let lambda_response: LeafSearchResponsePayload = - serde_json::from_slice(response_payload.as_ref())?; + serde_json::from_slice(response_payload.as_ref()) + .map_err(|e| SearchError::Internal(format!("JSON deserialization error: {}", e)))?; let response_bytes = BASE64_STANDARD .decode(&lambda_response.payload) - .map_err(|e| LambdaError::Serialization(format!("Base64 decode error: {}", e)))?; + .map_err(|e| SearchError::Internal(format!("Base64 decode error: {}", e)))?; - let leaf_response = LeafSearchResponse::decode(&response_bytes[..])?; + let leaf_response = LeafSearchResponse::decode(&response_bytes[..]) + .map_err(|e| SearchError::Internal(format!("Protobuf decode error: {}", e)))?; debug!( num_hits = leaf_response.num_hits, @@ -135,20 +138,4 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { Ok(leaf_response) } - - async fn health_check(&self) -> LambdaResult<()> { - // Try to get function configuration to verify it exists - let mut get_function = self.client.get_function().function_name(&self.function_name); - - if let Some(qualifier) = &self.qualifier { - get_function = get_function.qualifier(qualifier); - } - - get_function - .send() - .await - .map_err(|e| LambdaError::Configuration(format!("Lambda health check failed: {}", e)))?; - - Ok(()) - } } diff --git a/quickwit/quickwit-lambda/src/invoker/mod.rs b/quickwit/quickwit-lambda/src/invoker/mod.rs index 3cfd6d6f2f3..94061942cd9 100644 --- a/quickwit/quickwit-lambda/src/invoker/mod.rs +++ b/quickwit/quickwit-lambda/src/invoker/mod.rs @@ -12,32 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Remote function invoker abstraction for serverless leaf search execution. +//! AWS Lambda implementation of RemoteFunctionInvoker. mod aws_lambda; pub use aws_lambda::AwsLambdaInvoker; - -use async_trait::async_trait; -use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; - -use crate::error::LambdaResult; - -/// Trait for invoking remote serverless functions for leaf search. -/// -/// This abstraction allows different cloud providers to be supported -/// (AWS Lambda, GCP Cloud Functions, Azure Functions, etc.) -#[async_trait] -pub trait RemoteFunctionInvoker: Send + Sync + 'static { - /// Invoke the remote function with a LeafSearchRequest. - /// - /// The request is serialized to protobuf, sent to the function, - /// and the response is deserialized from protobuf. - async fn invoke_leaf_search( - &self, - request: LeafSearchRequest, - ) -> LambdaResult; - - /// Check if the invoker is properly configured and available. - async fn health_check(&self) -> LambdaResult<()>; -} +// Re-export the trait from quickwit-search +pub use quickwit_search::RemoteFunctionInvoker; diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda/src/lib.rs index fc7732c70f0..224b12c024b 100644 --- a/quickwit/quickwit-lambda/src/lib.rs +++ b/quickwit/quickwit-lambda/src/lib.rs @@ -16,8 +16,7 @@ //! //! This crate provides: //! - A Lambda handler that executes leaf search requests -//! - A `RemoteFunctionInvoker` trait for invoking serverless functions -//! - An AWS Lambda implementation of the invoker +//! - An AWS Lambda implementation of the `RemoteFunctionInvoker` trait mod config; mod context; @@ -25,8 +24,10 @@ mod error; mod handler; pub mod invoker; -pub use config::LambdaConfig; +pub use config::{LambdaConfig, LambdaSearcherConfig}; pub use context::LambdaSearcherContext; pub use error::{LambdaError, LambdaResult}; pub use handler::{handle_leaf_search, LeafSearchPayload, LeafSearchResponsePayload}; -pub use invoker::{AwsLambdaInvoker, RemoteFunctionInvoker}; +pub use invoker::AwsLambdaInvoker; +// Re-export RemoteFunctionInvoker trait from quickwit-search for convenience +pub use quickwit_search::RemoteFunctionInvoker; diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs index e1201ce7a0e..9ce1addf223 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs @@ -185,6 +185,11 @@ pub struct SearchRequest { /// that index is not found and this parameter is set to `false`. #[prost(bool, tag = "18")] pub ignore_missing_indexes: bool, + /// Execution mode for leaf search operations. + /// When set to EXECUTION_MODE_REMOTE_FUNCTION, leaf search is executed via + /// serverless functions (e.g., AWS Lambda) instead of gRPC to searcher nodes. + #[prost(enumeration = "ExecutionMode", tag = "19")] + pub execution_mode: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] @@ -615,6 +620,37 @@ impl ListFieldType { } } } +/// Execution mode for leaf search operations. +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[serde(rename_all = "snake_case")] +#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] +#[repr(i32)] +pub enum ExecutionMode { + /// Default: use gRPC to searcher nodes (existing behavior). + Grpc = 0, + /// Execute leaf search via remote serverless functions (e.g., AWS Lambda). + RemoteFunction = 1, +} +impl ExecutionMode { + /// String value of the enum field names used in the ProtoBuf definition. + /// + /// The values are not transformed in any way and thus are considered stable + /// (if the ProtoBuf definition does not change) and safe for programmatic use. + pub fn as_str_name(&self) -> &'static str { + match self { + Self::Grpc => "EXECUTION_MODE_GRPC", + Self::RemoteFunction => "EXECUTION_MODE_REMOTE_FUNCTION", + } + } + /// Creates an enum from field names used in the ProtoBuf definition. + pub fn from_str_name(value: &str) -> ::core::option::Option { + match value { + "EXECUTION_MODE_GRPC" => Some(Self::Grpc), + "EXECUTION_MODE_REMOTE_FUNCTION" => Some(Self::RemoteFunction), + _ => None, + } + } +} #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 213c1a6a90e..913b956f624 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -29,6 +29,7 @@ mod list_fields; mod list_fields_cache; mod list_terms; mod metrics_trackers; +mod remote_function; mod retry; mod root; mod scroll_context; @@ -79,6 +80,7 @@ pub use crate::client::{ }; pub use crate::cluster_client::ClusterClient; pub use crate::error::{SearchError, parse_grpc_error}; +pub use crate::remote_function::RemoteFunctionInvoker; use crate::fetch_docs::fetch_docs; pub use crate::root::{ IndexMetasForLeafSearch, SearchJob, check_all_index_metadata_found, jobs_to_leaf_request, @@ -298,6 +300,7 @@ pub async fn single_node_search( search_request, metastore, &cluster_client, + None, // No Lambda for single node search ) .await } diff --git a/quickwit/quickwit-search/src/remote_function.rs b/quickwit/quickwit-search/src/remote_function.rs new file mode 100644 index 00000000000..75da7a29207 --- /dev/null +++ b/quickwit/quickwit-search/src/remote_function.rs @@ -0,0 +1,33 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Trait for invoking remote serverless functions for leaf search. + +use async_trait::async_trait; +use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; + +use crate::SearchError; + +/// Trait for invoking remote serverless functions (e.g., AWS Lambda) for leaf search. +/// +/// This abstraction allows different cloud providers to be supported. +/// Implementations are provided by the `quickwit-lambda` crate. +#[async_trait] +pub trait RemoteFunctionInvoker: Send + Sync + 'static { + /// Invoke the remote function with a LeafSearchRequest. + async fn invoke_leaf_search( + &self, + request: LeafSearchRequest, + ) -> Result; +} diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 581665fa14f..8494fc9761c 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; +use std::sync::Arc; use std::sync::OnceLock; use std::sync::atomic::{AtomicU64, Ordering}; use std::time::{Duration, Instant}; @@ -31,9 +32,10 @@ use quickwit_proto::metastore::{ ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, }; use quickwit_proto::search::{ - FetchDocsRequest, FetchDocsResponse, Hit, LeafHit, LeafRequestRef, LeafSearchRequest, - LeafSearchResponse, PartialHit, SearchPlanResponse, SearchRequest, SearchResponse, - SnippetRequest, SortDatetimeFormat, SortField, SortValue, SplitIdAndFooterOffsets, + ExecutionMode, FetchDocsRequest, FetchDocsResponse, Hit, LeafHit, LeafRequestRef, + LeafSearchRequest, LeafSearchResponse, PartialHit, SearchPlanResponse, SearchRequest, + SearchResponse, SnippetRequest, SortDatetimeFormat, SortField, SortValue, + SplitIdAndFooterOffsets, }; use quickwit_proto::types::{IndexUid, SplitId}; use quickwit_query::query_ast::{ @@ -54,6 +56,7 @@ use crate::scroll_context::{ScrollContext, ScrollKeyAndStartOffset}; use crate::search_job_placer::{Job, group_by, group_jobs_by_index_id}; use crate::search_response_rest::StorageRequestCount; use crate::service::SearcherContext; +use crate::remote_function::RemoteFunctionInvoker; use crate::{ SearchError, SearchJobPlacer, SearchPlanResponseRest, SearchServiceClient, extract_split_and_footer_offsets, list_relevant_splits, @@ -372,6 +375,8 @@ fn simplify_search_request_for_scroll_api(req: &SearchRequest) -> crate::Result< // to recompute it afterward. count_hits: quickwit_proto::search::CountHits::Underestimate as i32, ignore_missing_indexes: req.ignore_missing_indexes, + // Scroll requests don't support Lambda execution + execution_mode: ExecutionMode::Grpc as i32, }) } @@ -568,6 +573,7 @@ async fn search_partial_hits_phase_with_scroll( mut search_request: SearchRequest, split_metadatas: &[SplitMetadata], cluster_client: &ClusterClient, + lambda_invoker: Option>, ) -> crate::Result<(LeafSearchResponse, Option)> { let scroll_ttl_opt = get_scroll_ttl_duration(&search_request)?; @@ -586,6 +592,7 @@ async fn search_partial_hits_phase_with_scroll( &search_request, split_metadatas, cluster_client, + lambda_invoker.clone(), ) .await?; let cached_partial_hits = leaf_search_resp.partial_hits.clone(); @@ -631,6 +638,7 @@ async fn search_partial_hits_phase_with_scroll( &search_request, split_metadatas, cluster_client, + lambda_invoker, ) .await?; Ok((leaf_search_resp, None)) @@ -735,10 +743,23 @@ pub(crate) async fn search_partial_hits_phase( search_request: &SearchRequest, split_metadatas: &[SplitMetadata], cluster_client: &ClusterClient, + lambda_invoker: Option>, ) -> crate::Result { let leaf_search_responses: Vec = if is_metadata_count_request(search_request) { get_count_from_metadata(split_metadatas) + } else if search_request.execution_mode == ExecutionMode::RemoteFunction as i32 + && lambda_invoker.is_some() + { + // Execute via Lambda + execute_leaf_search_via_lambda( + search_request, + indexes_metas_for_leaf_search, + split_metadatas, + lambda_invoker.as_ref().unwrap().as_ref(), + &searcher_context.searcher_config.lambda, + ) + .await? } else { let jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); let assigned_leaf_search_jobs = cluster_client @@ -805,6 +826,49 @@ pub(crate) async fn search_partial_hits_phase( Ok(leaf_search_response) } +/// Execute leaf search operations via remote serverless functions (e.g., AWS Lambda). +/// +/// This function batches splits into groups and invokes Lambda functions in parallel. +/// The batch size is controlled by `LambdaConfig::max_splits_per_invocation`. +async fn execute_leaf_search_via_lambda( + search_request: &SearchRequest, + indexes_metas_for_leaf_search: &IndexesMetasForLeafSearch, + split_metadatas: &[SplitMetadata], + lambda_invoker: &dyn RemoteFunctionInvoker, + lambda_config: &quickwit_config::LambdaConfig, +) -> crate::Result> { + let jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); + + // Batch splits for Lambda invocations + let batch_size = lambda_config.max_splits_per_invocation; + let batches: Vec> = jobs + .chunks(batch_size) + .map(|chunk| chunk.to_vec()) + .collect(); + + debug!( + num_splits = split_metadatas.len(), + num_batches = batches.len(), + batch_size, + "Executing leaf search via Lambda" + ); + + // Invoke Lambda functions in parallel for each batch + let lambda_tasks: Vec<_> = batches + .into_iter() + .map(|batch_jobs| { + let request = jobs_to_leaf_request(search_request, indexes_metas_for_leaf_search, batch_jobs); + async move { + let leaf_request = request?; + lambda_invoker.invoke_leaf_search(leaf_request).await + } + }) + .collect(); + + let responses = try_join_all(lambda_tasks).await?; + Ok(responses) +} + pub(crate) fn get_snippet_request(search_request: &SearchRequest) -> Option { if search_request.snippet_fields.is_empty() { return None; @@ -975,6 +1039,7 @@ async fn root_search_aux( search_request: SearchRequest, split_metadatas: Vec, cluster_client: &ClusterClient, + lambda_invoker: Option>, ) -> crate::Result { debug!(split_metadatas = ?PrettySample::new(&split_metadatas, 5)); let (first_phase_result, scroll_key_and_start_offset_opt): ( @@ -986,6 +1051,7 @@ async fn root_search_aux( search_request.clone(), &split_metadatas[..], cluster_client, + lambda_invoker, ) .await?; @@ -1204,6 +1270,7 @@ pub async fn root_search( mut search_request: SearchRequest, mut metastore: MetastoreServiceClient, cluster_client: &ClusterClient, + lambda_invoker: Option>, ) -> crate::Result { let start_instant = Instant::now(); @@ -1253,6 +1320,7 @@ pub async fn root_search( search_request, split_metadatas, cluster_client, + lambda_invoker, ), is_success: None, step: RootSearchMetricsStep::Exec { @@ -2666,7 +2734,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -2736,7 +2804,7 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -2828,7 +2896,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -2912,7 +2980,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -3044,7 +3112,7 @@ mod tests { &SearcherContext::for_test(), search_request.clone(), MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await?; @@ -3226,7 +3294,7 @@ mod tests { &SearcherContext::for_test(), search_request.clone(), MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await?; @@ -3350,7 +3418,7 @@ mod tests { &searcher_context, search_request, mock_metastore_client.clone(), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -3369,7 +3437,7 @@ mod tests { &searcher_context, search_request, mock_metastore_client, - &cluster_client, + &cluster_client, None, ) .await .unwrap_err(); @@ -3494,7 +3562,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -3634,7 +3702,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -3715,7 +3783,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -3782,7 +3850,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -3872,7 +3940,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -3954,7 +4022,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -4003,7 +4071,7 @@ mod tests { ..Default::default() }, metastore.clone(), - &cluster_client, + &cluster_client, None, ) .await .is_err() @@ -4019,7 +4087,7 @@ mod tests { ..Default::default() }, metastore, - &cluster_client, + &cluster_client, None, ) .await .is_err() @@ -4084,7 +4152,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await; assert!(search_response.is_err()); @@ -4134,7 +4202,7 @@ mod tests { &SearcherContext::for_test(), search_request, metastore.clone(), - &cluster_client, + &cluster_client, None, ) .await; assert!(search_response.is_err()); @@ -4154,7 +4222,7 @@ mod tests { &SearcherContext::for_test(), search_request, metastore, - &cluster_client, + &cluster_client, None, ) .await; assert!(search_response.is_err()); @@ -4672,7 +4740,7 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -4938,7 +5006,7 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -5120,7 +5188,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -5241,7 +5309,7 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap(); @@ -5294,7 +5362,7 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, + &cluster_client, None, ) .await .unwrap_err(); diff --git a/quickwit/quickwit-search/src/scroll_context.rs b/quickwit/quickwit-search/src/scroll_context.rs index a4a31a856b5..949e23a65a2 100644 --- a/quickwit/quickwit-search/src/scroll_context.rs +++ b/quickwit/quickwit-search/src/scroll_context.rs @@ -113,6 +113,7 @@ impl ScrollContext { &self.search_request, &self.split_metadatas[..], cluster_client, + None, // Lambda not supported for scroll queries ) .await?; self.cached_partial_hits_start_offset = start_offset; diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index cf2c0699b81..00c576c2544 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -167,6 +167,7 @@ impl SearchService for SearchServiceImpl { search_request, self.metastore.clone(), &self.cluster_client, + None, // TODO: Initialize lambda_invoker when Lambda is enabled ) .await?; Ok(search_result) diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs index db14c37700a..a605f3ea7f3 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs @@ -412,6 +412,7 @@ fn build_request_for_es_api( search_after, count_hits, ignore_missing_indexes, + execution_mode: 0, // Default to gRPC }, has_doc_id_field, )) diff --git a/quickwit/quickwit-serve/src/search_api/rest_handler.rs b/quickwit/quickwit-serve/src/search_api/rest_handler.rs index cfdf46c61ef..e81f80d483b 100644 --- a/quickwit/quickwit-serve/src/search_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/search_api/rest_handler.rs @@ -265,6 +265,7 @@ pub fn search_request_from_api_request( search_after: None, count_hits: search_request.count_all.into(), ignore_missing_indexes: false, + execution_mode: 0, // Default to gRPC }; Ok(search_request) } From b439914af19811c1e334546c0da3f3571ea98aa7 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 11:06:00 +0100 Subject: [PATCH 03/22] Summary of Changes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Key Technical Components 1. ExecutionMode enum in protobuf - Added GRPC and REMOTE_FUNCTION variants to SearchRequest 2. RemoteFunctionInvoker trait - Cloud-agnostic interface for function invocation 3. AwsLambdaInvoker - AWS Lambda implementation using the Lambda SDK 4. LambdaConfig - Configuration (enabled, function_name, max_splits_per_invocation, etc.) Files Modified File: quickwit-search/src/root.rs Changes: Lambda routing logic in search_partial_hits_phase(), added execute_leaf_search_via_lambda() function ──────────────────────────────────────── File: quickwit-search/src/service.rs Changes: Added lambda_invoker field to SearchServiceImpl ──────────────────────────────────────── File: quickwit-search/src/lib.rs Changes: Updated start_searcher_service to accept lambda_invoker ──────────────────────────────────────── File: quickwit-serve/src/lib.rs Changes: Initialize AwsLambdaInvoker when Lambda is enabled ──────────────────────────────────────── File: quickwit-serve/Cargo.toml Changes: Added quickwit-lambda dependency ──────────────────────────────────────── File: quickwit-lambda/* Changes: New crate with invoker, handler, and config Architecture - Lambda routing logic placed in search_partial_hits_phase() function - Split batching for Lambda invocations based on max_splits_per_invocation - Lambda invoker passed through: root_search → root_search_aux → search_partial_hits_phase_with_scroll → search_partial_hits_phase - Base64-encoded protobuf payloads for Lambda communication Lambda Configuration The Lambda feature is configured in the searcher section of your Quickwit node config: searcher: lambda: # Enable Lambda execution mode for leaf search enabled: true # AWS Lambda function name or ARN (required when enabled) function_name: "quickwit-leaf-search" # Optional function qualifier (alias or version) function_qualifier: "prod" # optional # Maximum number of splits per Lambda invocation (default: 10) max_splits_per_invocation: 10 # Timeout for Lambda invocations in seconds (default: 60) invocation_timeout_secs: 60 # Maximum number of concurrent Lambda invocations (default: 100) max_concurrent_invocations: 100 Triggering Lambda Mode To use Lambda for a search request, set execution_mode to REMOTE_FUNCTION in the SearchRequest protobuf, or use the appropriate query parameter/header in the REST API. Lambda Handler The Lambda function should be built from the quickwit-lambda crate's leaf_search binary, which: - Receives base64-encoded protobuf LeafSearchRequest - Returns base64-encoded protobuf LeafSearchResponse - Auto-configures cache sizes based on Lambda memory allocation --- quickwit/Cargo.lock | 1 + .../quickwit-lambda/src/bin/leaf_search.rs | 22 ++--- quickwit/quickwit-lambda/src/config.rs | 1 - .../quickwit-lambda/src/invoker/aws_lambda.rs | 2 +- quickwit/quickwit-lambda/src/lib.rs | 2 +- quickwit/quickwit-search/src/lib.rs | 5 +- quickwit/quickwit-search/src/root.rs | 80 ++++++++++++------- quickwit/quickwit-search/src/service.rs | 6 +- quickwit/quickwit-serve/Cargo.toml | 1 + quickwit/quickwit-serve/src/lib.rs | 19 ++++- 10 files changed, 94 insertions(+), 45 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 2a6628dbc1c..885882dbbf8 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7632,6 +7632,7 @@ dependencies = [ "quickwit-ingest", "quickwit-jaeger", "quickwit-janitor", + "quickwit-lambda", "quickwit-metastore", "quickwit-opentelemetry", "quickwit-proto", diff --git a/quickwit/quickwit-lambda/src/bin/leaf_search.rs b/quickwit/quickwit-lambda/src/bin/leaf_search.rs index ef137f33293..504745dbdf6 100644 --- a/quickwit/quickwit-lambda/src/bin/leaf_search.rs +++ b/quickwit/quickwit-lambda/src/bin/leaf_search.rs @@ -14,8 +14,8 @@ //! AWS Lambda binary entry point for Quickwit leaf search. -use lambda_runtime::{service_fn, Error, LambdaEvent}; -use quickwit_lambda::{handle_leaf_search, LambdaSearcherContext, LeafSearchPayload}; +use lambda_runtime::{Error, LambdaEvent, service_fn}; +use quickwit_lambda::{LambdaSearcherContext, LeafSearchPayload, handle_leaf_search}; use tracing::info; use tracing_subscriber::EnvFilter; @@ -27,19 +27,21 @@ async fn main() -> Result<(), Error> { .json() .init(); - info!("Starting Quickwit Lambda leaf search handler"); + info!("starting Quickwit Lambda leaf search handler"); // Pre-initialize context on cold start let context = LambdaSearcherContext::get_or_init().await; - info!("Lambda context initialized, starting handler loop"); + info!("lambda context initialized, starting handler loop"); // Run the Lambda handler - lambda_runtime::run(service_fn(|event: LambdaEvent| async move { - let (payload, _lambda_ctx) = event.into_parts(); - handle_leaf_search(payload, context) - .await - .map_err(|e| lambda_runtime::Error::from(e.to_string())) - })) + lambda_runtime::run(service_fn( + |event: LambdaEvent| async move { + let (payload, _lambda_ctx) = event.into_parts(); + handle_leaf_search(payload, context) + .await + .map_err(|e| lambda_runtime::Error::from(e.to_string())) + }, + )) .await } diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda/src/config.rs index 93fd7d146c1..d92196f4492 100644 --- a/quickwit/quickwit-lambda/src/config.rs +++ b/quickwit/quickwit-lambda/src/config.rs @@ -13,7 +13,6 @@ // limitations under the License. use bytesize::ByteSize; - // Re-export LambdaConfig from quickwit-config pub use quickwit_config::LambdaConfig; diff --git a/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs b/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs index 1f78b050660..0e6a1696978 100644 --- a/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs +++ b/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs @@ -13,9 +13,9 @@ // limitations under the License. use async_trait::async_trait; +use aws_sdk_lambda::Client as LambdaClient; use aws_sdk_lambda::primitives::Blob; use aws_sdk_lambda::types::InvocationType; -use aws_sdk_lambda::Client as LambdaClient; use base64::prelude::*; use prost::Message; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda/src/lib.rs index 224b12c024b..32c0fa49286 100644 --- a/quickwit/quickwit-lambda/src/lib.rs +++ b/quickwit/quickwit-lambda/src/lib.rs @@ -27,7 +27,7 @@ pub mod invoker; pub use config::{LambdaConfig, LambdaSearcherConfig}; pub use context::LambdaSearcherContext; pub use error::{LambdaError, LambdaResult}; -pub use handler::{handle_leaf_search, LeafSearchPayload, LeafSearchResponsePayload}; +pub use handler::{LeafSearchPayload, LeafSearchResponsePayload, handle_leaf_search}; pub use invoker::AwsLambdaInvoker; // Re-export RemoteFunctionInvoker trait from quickwit-search for convenience pub use quickwit_search::RemoteFunctionInvoker; diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 913b956f624..6f12272a204 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -80,8 +80,8 @@ pub use crate::client::{ }; pub use crate::cluster_client::ClusterClient; pub use crate::error::{SearchError, parse_grpc_error}; -pub use crate::remote_function::RemoteFunctionInvoker; use crate::fetch_docs::fetch_docs; +pub use crate::remote_function::RemoteFunctionInvoker; pub use crate::root::{ IndexMetasForLeafSearch, SearchJob, check_all_index_metadata_found, jobs_to_leaf_request, root_search, search_plan, @@ -262,6 +262,7 @@ pub async fn start_searcher_service( storage_resolver: StorageResolver, search_job_placer: SearchJobPlacer, searcher_context: Arc, + lambda_invoker: Option>, ) -> anyhow::Result> { let cluster_client = ClusterClient::new(search_job_placer); let search_service = Arc::new(SearchServiceImpl::new( @@ -269,6 +270,7 @@ pub async fn start_searcher_service( storage_resolver, cluster_client, searcher_context, + lambda_invoker, )); Ok(search_service) } @@ -291,6 +293,7 @@ pub async fn single_node_search( storage_resolver, cluster_client.clone(), searcher_context.clone(), + None, // No Lambda for single node search )); let search_service_client = SearchServiceClient::from_service(search_service.clone(), socket_addr); diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 8494fc9761c..0f1a698ad42 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -13,9 +13,8 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; -use std::sync::Arc; -use std::sync::OnceLock; use std::sync::atomic::{AtomicU64, Ordering}; +use std::sync::{Arc, OnceLock}; use std::time::{Duration, Instant}; use anyhow::Context; @@ -52,11 +51,11 @@ use tracing::{debug, info, info_span, instrument}; use crate::cluster_client::ClusterClient; use crate::collector::{QuickwitAggregations, make_merge_collector}; use crate::metrics_trackers::{RootSearchMetricsFuture, RootSearchMetricsStep}; +use crate::remote_function::RemoteFunctionInvoker; use crate::scroll_context::{ScrollContext, ScrollKeyAndStartOffset}; use crate::search_job_placer::{Job, group_by, group_jobs_by_index_id}; use crate::search_response_rest::StorageRequestCount; use crate::service::SearcherContext; -use crate::remote_function::RemoteFunctionInvoker; use crate::{ SearchError, SearchJobPlacer, SearchPlanResponseRest, SearchServiceClient, extract_split_and_footer_offsets, list_relevant_splits, @@ -857,7 +856,8 @@ async fn execute_leaf_search_via_lambda( let lambda_tasks: Vec<_> = batches .into_iter() .map(|batch_jobs| { - let request = jobs_to_leaf_request(search_request, indexes_metas_for_leaf_search, batch_jobs); + let request = + jobs_to_leaf_request(search_request, indexes_metas_for_leaf_search, batch_jobs); async move { let leaf_request = request?; lambda_invoker.invoke_leaf_search(leaf_request).await @@ -2734,7 +2734,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -2804,7 +2805,8 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -2896,7 +2898,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -2980,7 +2983,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -3112,7 +3116,8 @@ mod tests { &SearcherContext::for_test(), search_request.clone(), MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await?; @@ -3294,7 +3299,8 @@ mod tests { &SearcherContext::for_test(), search_request.clone(), MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await?; @@ -3418,7 +3424,8 @@ mod tests { &searcher_context, search_request, mock_metastore_client.clone(), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -3437,7 +3444,8 @@ mod tests { &searcher_context, search_request, mock_metastore_client, - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap_err(); @@ -3562,7 +3570,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -3702,7 +3711,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -3783,7 +3793,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -3850,7 +3861,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -3940,7 +3952,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -4022,7 +4035,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -4071,7 +4085,8 @@ mod tests { ..Default::default() }, metastore.clone(), - &cluster_client, None, + &cluster_client, + None, ) .await .is_err() @@ -4087,7 +4102,8 @@ mod tests { ..Default::default() }, metastore, - &cluster_client, None, + &cluster_client, + None, ) .await .is_err() @@ -4152,7 +4168,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await; assert!(search_response.is_err()); @@ -4202,7 +4219,8 @@ mod tests { &SearcherContext::for_test(), search_request, metastore.clone(), - &cluster_client, None, + &cluster_client, + None, ) .await; assert!(search_response.is_err()); @@ -4222,7 +4240,8 @@ mod tests { &SearcherContext::for_test(), search_request, metastore, - &cluster_client, None, + &cluster_client, + None, ) .await; assert!(search_response.is_err()); @@ -4740,7 +4759,8 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -5006,7 +5026,8 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -5188,7 +5209,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -5309,7 +5331,8 @@ mod tests { &SearcherContext::for_test(), search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap(); @@ -5362,7 +5385,8 @@ mod tests { &searcher_context, search_request, MetastoreServiceClient::from_mock(mock_metastore), - &cluster_client, None, + &cluster_client, + None, ) .await .unwrap_err(); diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index 00c576c2544..851358ee965 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -39,6 +39,7 @@ use crate::list_fields::{leaf_list_fields, root_list_fields}; use crate::list_fields_cache::ListFieldsCache; use crate::list_terms::{leaf_list_terms, root_list_terms}; use crate::metrics_trackers::LeafSearchMetricsFuture; +use crate::remote_function::RemoteFunctionInvoker; use crate::root::fetch_docs_phase; use crate::scroll_context::{MiniKV, ScrollContext, ScrollKeyAndStartOffset}; use crate::search_permit_provider::SearchPermitProvider; @@ -52,6 +53,7 @@ pub struct SearchServiceImpl { cluster_client: ClusterClient, searcher_context: Arc, local_kv_store: MiniKV, + lambda_invoker: Option>, } /// Trait representing a search service. @@ -141,6 +143,7 @@ impl SearchServiceImpl { storage_resolver: StorageResolver, cluster_client: ClusterClient, searcher_context: Arc, + lambda_invoker: Option>, ) -> Self { SearchServiceImpl { metastore, @@ -148,6 +151,7 @@ impl SearchServiceImpl { cluster_client, searcher_context, local_kv_store: MiniKV::default(), + lambda_invoker, } } } @@ -167,7 +171,7 @@ impl SearchService for SearchServiceImpl { search_request, self.metastore.clone(), &self.cluster_client, - None, // TODO: Initialize lambda_invoker when Lambda is enabled + self.lambda_invoker.clone(), ) .await?; Ok(search_result) diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index 363065a3403..eadc2fb8c80 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -72,6 +72,7 @@ quickwit-opentelemetry = { workspace = true } quickwit-proto = { workspace = true } quickwit-query = { workspace = true } quickwit-search = { workspace = true } +quickwit-lambda = { workspace = true } quickwit-storage = { workspace = true } quickwit-telemetry = { workspace = true } diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index ca4520ff0ce..cef58905695 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -87,6 +87,7 @@ use quickwit_ingest::{ }; use quickwit_jaeger::JaegerService; use quickwit_janitor::{JanitorService, start_janitor_service}; +use quickwit_lambda::AwsLambdaInvoker; use quickwit_metastore::{ ControlPlaneMetastore, ListIndexesMetadataResponseExt, MetastoreResolver, }; @@ -106,8 +107,8 @@ use quickwit_proto::metastore::{ use quickwit_proto::search::ReportSplitsRequest; use quickwit_proto::types::NodeId; use quickwit_search::{ - SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool, - create_search_client_from_channel, start_searcher_service, + RemoteFunctionInvoker, SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, + SearcherPool, create_search_client_from_channel, start_searcher_service, }; use quickwit_storage::{SplitCache, StorageResolver}; use tcp_listener::TcpListenerResolver; @@ -1017,11 +1018,25 @@ async fn setup_searcher( ) -> anyhow::Result<(SearchJobPlacer, Arc)> { let searcher_pool = SearcherPool::default(); let search_job_placer = SearchJobPlacer::new(searcher_pool.clone()); + + // Initialize Lambda invoker if enabled + let lambda_invoker: Option> = + if node_config.searcher_config.lambda.enabled { + info!("initializing AWS Lambda invoker for leaf search"); + let invoker = AwsLambdaInvoker::new(&node_config.searcher_config.lambda) + .await + .context("failed to initialize AWS Lambda invoker")?; + Some(Arc::new(invoker)) + } else { + None + }; + let search_service = start_searcher_service( metastore, storage_resolver, search_job_placer.clone(), searcher_context, + lambda_invoker, ) .await?; let search_service_clone = search_service.clone(); From 63818cda2511958a4b89479e12b2775e001f7f94 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 11:52:02 +0100 Subject: [PATCH 04/22] File rationalization --- .../src/{invoker/aws_lambda.rs => invoker.rs} | 0 quickwit/quickwit-lambda/src/invoker/mod.rs | 21 ------------------- quickwit/quickwit-lambda/src/lib.rs | 4 +--- 3 files changed, 1 insertion(+), 24 deletions(-) rename quickwit/quickwit-lambda/src/{invoker/aws_lambda.rs => invoker.rs} (100%) delete mode 100644 quickwit/quickwit-lambda/src/invoker/mod.rs diff --git a/quickwit/quickwit-lambda/src/invoker/aws_lambda.rs b/quickwit/quickwit-lambda/src/invoker.rs similarity index 100% rename from quickwit/quickwit-lambda/src/invoker/aws_lambda.rs rename to quickwit/quickwit-lambda/src/invoker.rs diff --git a/quickwit/quickwit-lambda/src/invoker/mod.rs b/quickwit/quickwit-lambda/src/invoker/mod.rs deleted file mode 100644 index 94061942cd9..00000000000 --- a/quickwit/quickwit-lambda/src/invoker/mod.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Copyright 2021-Present Datadog, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! AWS Lambda implementation of RemoteFunctionInvoker. - -mod aws_lambda; - -pub use aws_lambda::AwsLambdaInvoker; -// Re-export the trait from quickwit-search -pub use quickwit_search::RemoteFunctionInvoker; diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda/src/lib.rs index 32c0fa49286..ea235403c11 100644 --- a/quickwit/quickwit-lambda/src/lib.rs +++ b/quickwit/quickwit-lambda/src/lib.rs @@ -22,12 +22,10 @@ mod config; mod context; mod error; mod handler; -pub mod invoker; +mod invoker; pub use config::{LambdaConfig, LambdaSearcherConfig}; pub use context::LambdaSearcherContext; pub use error::{LambdaError, LambdaResult}; pub use handler::{LeafSearchPayload, LeafSearchResponsePayload, handle_leaf_search}; pub use invoker::AwsLambdaInvoker; -// Re-export RemoteFunctionInvoker trait from quickwit-search for convenience -pub use quickwit_search::RemoteFunctionInvoker; From 8910340d922011bdb894e9d4dc9a96e5bc2ca4aa Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 11:54:46 +0100 Subject: [PATCH 05/22] blop --- quickwit/quickwit-lambda/src/bin/leaf_search.rs | 2 -- quickwit/quickwit-lambda/src/handler.rs | 6 +++--- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/quickwit/quickwit-lambda/src/bin/leaf_search.rs b/quickwit/quickwit-lambda/src/bin/leaf_search.rs index 504745dbdf6..4f8412534fb 100644 --- a/quickwit/quickwit-lambda/src/bin/leaf_search.rs +++ b/quickwit/quickwit-lambda/src/bin/leaf_search.rs @@ -27,8 +27,6 @@ async fn main() -> Result<(), Error> { .json() .init(); - info!("starting Quickwit Lambda leaf search handler"); - // Pre-initialize context on cold start let context = LambdaSearcherContext::get_or_init().await; diff --git a/quickwit/quickwit-lambda/src/handler.rs b/quickwit/quickwit-lambda/src/handler.rs index b22f21d7415..41399021a9a 100644 --- a/quickwit/quickwit-lambda/src/handler.rs +++ b/quickwit/quickwit-lambda/src/handler.rs @@ -53,10 +53,10 @@ pub async fn handle_leaf_search( let num_splits: usize = leaf_search_request .leaf_requests .iter() - .map(|r| r.split_offsets.len()) + .map(|leaf_request_ref| leaf_request_ref.split_offsets.len()) .sum(); - info!(num_splits, "Processing leaf search request"); + info!(num_splits, "processing leaf search request"); // Execute leaf search let leaf_search_response = multi_index_leaf_search( @@ -70,7 +70,7 @@ pub async fn handle_leaf_search( info!( num_hits = leaf_search_response.num_hits, num_successful_splits = leaf_search_response.num_successful_splits, - "Leaf search completed" + "leaf search completed" ); // Serialize response From 984b56c55d34ca85cafb3e3e6e02eb7eb8d9ecf5 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 12:12:23 +0100 Subject: [PATCH 06/22] lambda function validation --- quickwit/quickwit-lambda/src/invoker.rs | 23 +++++++++++++++++++++++ quickwit/quickwit-serve/src/lib.rs | 7 +++++++ 2 files changed, 30 insertions(+) diff --git a/quickwit/quickwit-lambda/src/invoker.rs b/quickwit/quickwit-lambda/src/invoker.rs index 0e6a1696978..2c2d58001c6 100644 --- a/quickwit/quickwit-lambda/src/invoker.rs +++ b/quickwit/quickwit-lambda/src/invoker.rs @@ -63,6 +63,29 @@ impl AwsLambdaInvoker { qualifier, } } + + /// Validate that the Lambda function exists and is invocable. + /// Uses DryRun invocation type - validates without executing. + pub async fn validate(&self) -> LambdaResult<()> { + let mut request = self + .client + .invoke() + .function_name(&self.function_name) + .invocation_type(InvocationType::DryRun); + + if let Some(qualifier) = &self.qualifier { + request = request.qualifier(qualifier); + } + + request.send().await.map_err(|e| { + LambdaError::Configuration(format!( + "Failed to validate Lambda function '{}': {}", + self.function_name, e + )) + })?; + + Ok(()) + } } #[async_trait] diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index cef58905695..dd12ea3637d 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1026,6 +1026,13 @@ async fn setup_searcher( let invoker = AwsLambdaInvoker::new(&node_config.searcher_config.lambda) .await .context("failed to initialize AWS Lambda invoker")?; + + // Validate function exists and is accessible + invoker + .validate() + .await + .context("Lambda function validation failed")?; + Some(Arc::new(invoker)) } else { None From fba9519c7da9d06cc7fbe6423a4bd7c775f30bd1 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 15:03:53 +0100 Subject: [PATCH 07/22] deployment --- quickwit/Cargo.lock | 13 + quickwit/Cargo.toml | 1 + .../quickwit-config/src/node_config/mod.rs | 31 ++- quickwit/quickwit-lambda/Cargo.toml | 7 + quickwit/quickwit-lambda/README.md | 225 +++++++++++++++++- quickwit/quickwit-lambda/src/config.rs | 16 -- quickwit/quickwit-lambda/src/context.rs | 5 +- quickwit/quickwit-lambda/src/error.rs | 9 + quickwit/quickwit-lambda/src/invoker.rs | 7 +- quickwit/quickwit-lambda/src/lib.rs | 3 + quickwit/quickwit-search/src/leaf.rs | 3 +- quickwit/quickwit-search/src/lib.rs | 2 + quickwit/quickwit-serve/src/lib.rs | 15 +- 13 files changed, 303 insertions(+), 34 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 885882dbbf8..7e6c433900a 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7375,6 +7375,7 @@ dependencies = [ "tokio", "tracing", "tracing-subscriber", + "zip", ] [[package]] @@ -11803,6 +11804,18 @@ dependencies = [ "syn 2.0.114", ] +[[package]] +name = "zip" +version = "0.6.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "760394e246e4c28189f19d488c058bf16f564016aefac5d32bb1f3b51d5e9261" +dependencies = [ + "byteorder", + "crc32fast", + "crossbeam-utils", + "flate2", +] + [[package]] name = "zlib-rs" version = "0.5.5" diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 7aa2445e98b..7bf6ad55fce 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -311,6 +311,7 @@ vrl = { version = "0.29", default-features = false, features = [ warp = { version = "0.4", features = ["server", "test"] } whichlang = "0.1" wiremock = "0.6" +zip = { version = "0.6", default-features = false, features = ["deflate"] } zstd = { version = "0.13", default-features = false } aws-config = "1.8" diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index bfd27984e70..238e237430e 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -313,8 +313,8 @@ pub struct LambdaConfig { #[serde(default)] pub enabled: bool, /// AWS Lambda function name or ARN. - #[serde(default)] - pub function_name: Option, + #[serde(default = "LambdaConfig::default_function_name")] + pub function_name: String, /// Optional function qualifier (alias or version). #[serde(default)] pub function_qualifier: Option, @@ -327,22 +327,41 @@ pub struct LambdaConfig { /// Maximum number of concurrent Lambda invocations. #[serde(default = "LambdaConfig::default_max_concurrent_invocations")] pub max_concurrent_invocations: usize, + /// Enable automatic Lambda function deployment at startup. + #[serde(default)] + pub auto_deploy: bool, + /// IAM execution role ARN for the Lambda function (required if auto_deploy=true). + #[serde(default)] + pub execution_role_arn: Option, + /// Memory size for the Lambda function in MB (default: 1024). + #[serde(default = "LambdaConfig::default_memory_size_mb")] + pub memory_size_mb: u32, + /// Timeout for the Lambda function in seconds (default: 30). + #[serde(default = "LambdaConfig::default_timeout_secs")] + pub timeout_secs: u32, } impl Default for LambdaConfig { fn default() -> Self { Self { enabled: false, - function_name: None, + function_name: Self::default_function_name(), function_qualifier: None, max_splits_per_invocation: Self::default_max_splits_per_invocation(), invocation_timeout_secs: Self::default_invocation_timeout_secs(), max_concurrent_invocations: Self::default_max_concurrent_invocations(), + auto_deploy: false, + execution_role_arn: None, + memory_size_mb: Self::default_memory_size_mb(), + timeout_secs: Self::default_timeout_secs(), } } } impl LambdaConfig { + fn default_function_name() -> String { + "quickwit-lambda-search".to_string() + } fn default_max_splits_per_invocation() -> usize { 10 } @@ -352,6 +371,12 @@ impl LambdaConfig { fn default_max_concurrent_invocations() -> usize { 100 } + fn default_memory_size_mb() -> u32 { + 1024 + } + fn default_timeout_secs() -> u32 { + 30 + } } #[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] diff --git a/quickwit/quickwit-lambda/Cargo.toml b/quickwit/quickwit-lambda/Cargo.toml index eb801dd20ce..bd1bb6359f8 100644 --- a/quickwit/quickwit-lambda/Cargo.toml +++ b/quickwit/quickwit-lambda/Cargo.toml @@ -44,6 +44,13 @@ tempfile = { workspace = true } name = "quickwit-lambda-leaf-search" path = "src/bin/leaf_search.rs" +[build-dependencies] +# Build dependencies for auto-deploy feature. +zip = { workspace = true, optional = true } + [features] default = [] testsuite = [] +# Enable automatic Lambda function deployment at startup. +# When enabled, the Lambda binary is embedded at compile time. +auto-deploy = ["dep:zip"] diff --git a/quickwit/quickwit-lambda/README.md b/quickwit/quickwit-lambda/README.md index 88fa9c8748a..5734c6cc55d 100644 --- a/quickwit/quickwit-lambda/README.md +++ b/quickwit/quickwit-lambda/README.md @@ -1,4 +1,223 @@ -# Deprecation +# Quickwit Lambda -This package was removed in Q3 2025. The maintenance burden was high and the -feature was unused. \ No newline at end of file +This crate provides AWS Lambda support for Quickwit's leaf search operations, enabling serverless execution of search queries across splits. + +## Overview + +Quickwit can offload leaf search operations to AWS Lambda functions. This is useful for: +- Scaling search capacity elastically +- Reducing infrastructure costs for sporadic workloads +- Isolating search execution from the main Quickwit cluster + +## Features + +### Lambda Invoker + +The `RemoteFunctionInvoker` trait is meant to allow implementing a custom invoker for other cloud providers, +but at the moment only AWS is supported. + +### Auto-Deploy (Optional) + +When the `auto-deploy` feature is enabled, Quickwit can automatically create or update the Lambda function at startup. This eliminates the need to manually deploy the Lambda function. + +## Configuration + +Add the following to your Quickwit configuration: + +```yaml +searcher: + lambda: + enabled: true + # Auto-deploy settings (requires auto-deploy feature) + auto_deploy: true + execution_role_arn: "arn:aws:iam::123456789:role/quickwit-lambda-role" +``` + +### Configuration Options + +| Option | Type | Default | Description | +|--------|------|---------|-------------| +| `enabled` | bool | `false` | Enable Lambda execution for leaf search | +| `function_name` | string | `"quickwit-lambda-search"` | Lambda function name or ARN | +| `function_qualifier` | string | `null` | Optional alias or version qualifier | +| `max_splits_per_invocation` | int | `10` | Maximum splits per Lambda invocation | +| `invocation_timeout_secs` | int | `30` | Client-side timeout for invocations | +| `max_concurrent_invocations` | int | `100` | Maximum concurrent Lambda invocations | +| `auto_deploy` | bool | `false` | Enable automatic Lambda deployment | +| `execution_role_arn` | string | `null` | IAM role ARN (required for auto-deploy) | +| `memory_size_mb` | int | `1024` | Lambda memory allocation in MB | +| `timeout_secs` | int | `30` | Lambda function timeout in seconds | + +## Auto-Deploy Feature + +The auto-deploy feature allows Quickwit to automatically manage the Lambda function lifecycle. + +### Build-Time Binary Packaging + +When building Quickwit with the `auto-deploy` feature enabled, the build script (`build.rs`) packages the Lambda binary into a zip file that gets embedded in the Quickwit binary: + +```bash +cargo build --release -p quickwit-lambda --features auto-deploy +``` + +The build script looks for the Lambda binary in the following locations (in order): + +1. `QUICKWIT_LAMBDA_BINARY_PATH` environment variable +2. `target/aarch64-unknown-linux-musl/release/quickwit-lambda-leaf-search` +3. `target/lambda/quickwit-lambda-leaf-search/bootstrap` +4. `lambda_bootstrap` in the workspace root + +If no binary is found, a placeholder is created. This allows development builds to succeed, but the placeholder will fail if actually deployed. + +**For production builds**, you must first cross-compile the Lambda binary for ARM64 Linux: + +```bash +# Install the target +rustup target add aarch64-unknown-linux-musl + +# Build the Lambda binary +cargo build --release --target aarch64-unknown-linux-musl -p quickwit-lambda --bin quickwit-lambda-leaf-search + +# Then build Quickwit with auto-deploy +cargo build --release --features auto-deploy +``` + +### Startup Behavior + +When `auto_deploy: true` is configured, the following happens at Quickwit startup: + +1. **Function Check**: Quickwit checks if the Lambda function exists +2. **Create or Update**: + - If the function doesn't exist, it creates it + - If the function exists but has a different version tag, it updates the code and configuration + - If the function exists with the same version, no action is taken +3. **Validation**: After deployment, Quickwit validates the function is invocable + +**Important**: If deployment fails for any reason, Quickwit startup fails immediately. This fail-fast behavior ensures you don't run a Quickwit cluster that expects Lambda execution but can't actually invoke the function. + +### Multi-Node Coordination + +The auto-deploy feature is safe for concurrent execution from multiple Quickwit nodes: + +- AWS Lambda's `CreateFunction` API is idempotent - if multiple nodes try to create the same function simultaneously, one succeeds and others receive a `ResourceConflictException` +- When a conflict is detected, the node falls back to updating the existing function +- Version tracking via tags ensures updates only happen when necessary + +### Version Management + +Quickwit tracks the deployed Lambda version using a `quickwit_version` tag on the function. On startup: + +- If the tag matches the current Quickwit version, no update is performed +- If the tag differs or is missing, the function is updated + +This means upgrading Quickwit automatically updates the Lambda function to match. + +## IAM Permissions + +### For the Quickwit Process (Auto-Deploy) + +The Quickwit process needs these permissions to auto-deploy: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Action": [ + "lambda:CreateFunction", + "lambda:UpdateFunctionCode", + "lambda:UpdateFunctionConfiguration", + "lambda:GetFunction", + "lambda:TagResource" + ], + "Resource": "arn:aws:lambda:*:*:function:quickwit-*" + }, + { + "Effect": "Allow", + "Action": "iam:PassRole", + "Resource": "arn:aws:iam::*:role/quickwit-lambda-*" + } + ] +} +``` + +### For Invoking the Lambda + +The Quickwit process needs permission to invoke the function: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Action": "lambda:InvokeFunction", + "Resource": "arn:aws:lambda:*:*:function:quickwit-*" + } + ] +} +``` + +### Lambda Execution Role + +The Lambda function itself needs an execution role with: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Action": [ + "logs:CreateLogGroup", + "logs:CreateLogStream", + "logs:PutLogEvents" + ], + "Resource": "arn:aws:logs:*:*:*" + }, + { + "Effect": "Allow", + "Action": [ + "s3:GetObject", + "s3:ListBucket" + ], + "Resource": [ + "arn:aws:s3:::your-index-bucket", + "arn:aws:s3:::your-index-bucket/*" + ] + } + ] +} +``` + +## Architecture + +The Lambda function: +- Runs on ARM64 (Graviton2) for cost efficiency +- Uses the `provided.al2023` runtime +- Receives protobuf-encoded `LeafSearchRequest` payloads (base64 in JSON) +- Returns protobuf-encoded `LeafSearchResponse` payloads (base64 in JSON) + +## Troubleshooting + +### Startup Failures + +If Quickwit fails to start with Lambda enabled: + +1. **"execution_role_arn required for auto_deploy"**: You enabled `auto_deploy` but didn't provide an IAM role ARN +2. **"failed to create function"**: Check IAM permissions for the Quickwit process +3. **"Lambda function validation failed"**: The function exists but isn't invocable - check the function's execution role + +### Runtime Issues + +1. **Timeouts**: Increase `invocation_timeout_secs` and/or `timeout_secs` +2. **Memory errors**: Increase `memory_size_mb` +3. **Throttling**: AWS Lambda has concurrency limits - check your account limits + +### Build Issues + +If you see "No Lambda binary found, creating placeholder zip": +- This is expected for development builds without cross-compilation +- For production, ensure the Lambda binary is built for `aarch64-unknown-linux-musl` +- Set `QUICKWIT_LAMBDA_BINARY_PATH` to point to your pre-built binary diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda/src/config.rs index d92196f4492..6178256ddfd 100644 --- a/quickwit/quickwit-lambda/src/config.rs +++ b/quickwit/quickwit-lambda/src/config.rs @@ -20,15 +20,6 @@ pub use quickwit_config::LambdaConfig; /// These settings are optimized for Lambda's memory constraints. #[derive(Clone, Debug)] pub struct LambdaSearcherConfig { - /// Memory allocated to the Lambda function in MB. - pub memory_mb: usize, - - /// Fast field cache capacity (derived from memory_mb). - pub fast_field_cache_capacity: ByteSize, - - /// Split footer cache capacity. - pub split_footer_cache_capacity: ByteSize, - /// Maximum concurrent split searches within a single Lambda invocation. pub max_concurrent_split_searches: usize, @@ -39,17 +30,10 @@ pub struct LambdaSearcherConfig { impl LambdaSearcherConfig { /// Create a Lambda-optimized searcher config based on the allocated memory. pub fn for_memory(memory_mb: usize) -> Self { - // Allocate roughly 1/4 of memory to fast field cache - let fast_field_cache_capacity = ByteSize::mb((memory_mb / 4) as u64); - // Fixed reasonable sizes for other caches - let split_footer_cache_capacity = ByteSize::mb(50); // Warmup budget is about half of memory let warmup_memory_budget = ByteSize::mb((memory_mb / 2) as u64); Self { - memory_mb, - fast_field_cache_capacity, - split_footer_cache_capacity, max_concurrent_split_searches: 20, warmup_memory_budget, } diff --git a/quickwit/quickwit-lambda/src/context.rs b/quickwit/quickwit-lambda/src/context.rs index 2272065a922..3bc00c17856 100644 --- a/quickwit/quickwit-lambda/src/context.rs +++ b/quickwit/quickwit-lambda/src/context.rs @@ -30,10 +30,7 @@ pub struct LambdaSearcherContext { impl LambdaSearcherContext { pub async fn new(config: LambdaSearcherConfig) -> LambdaResult { - info!( - memory_mb = config.memory_mb, - "Initializing Lambda searcher context" - ); + info!("Initializing Lambda searcher context"); let searcher_config = create_searcher_config(&config); let searcher_context = Arc::new(SearcherContext::new(searcher_config, None)); diff --git a/quickwit/quickwit-lambda/src/error.rs b/quickwit/quickwit-lambda/src/error.rs index a38f7774eb7..f40ff93a4b8 100644 --- a/quickwit/quickwit-lambda/src/error.rs +++ b/quickwit/quickwit-lambda/src/error.rs @@ -34,6 +34,12 @@ pub enum LambdaError { Configuration(String), /// Internal error. Internal(String), + /// Resource conflict (e.g., function already exists during concurrent create). + ResourceConflict, + /// Error during Lambda function deployment. + Deployment(String), + /// Lambda function not found. + NotFound(String), } impl fmt::Display for LambdaError { @@ -45,6 +51,9 @@ impl fmt::Display for LambdaError { LambdaError::FunctionError(msg) => write!(f, "Lambda function error: {}", msg), LambdaError::Configuration(msg) => write!(f, "Configuration error: {}", msg), LambdaError::Internal(msg) => write!(f, "Internal error: {}", msg), + LambdaError::ResourceConflict => write!(f, "Resource conflict: function already exists"), + LambdaError::Deployment(msg) => write!(f, "Deployment error: {}", msg), + LambdaError::NotFound(name) => write!(f, "Lambda function not found: {}", name), } } } diff --git a/quickwit/quickwit-lambda/src/invoker.rs b/quickwit/quickwit-lambda/src/invoker.rs index 2c2d58001c6..3e5e2499595 100644 --- a/quickwit/quickwit-lambda/src/invoker.rs +++ b/quickwit/quickwit-lambda/src/invoker.rs @@ -36,17 +36,12 @@ pub struct AwsLambdaInvoker { impl AwsLambdaInvoker { /// Create a new AWS Lambda invoker with the given configuration. pub async fn new(config: &LambdaConfig) -> LambdaResult { - let function_name = config - .function_name - .clone() - .ok_or_else(|| LambdaError::Configuration("Lambda function_name is required".into()))?; - let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; let client = LambdaClient::new(&aws_config); Ok(Self { client, - function_name, + function_name: config.function_name.clone(), qualifier: config.function_qualifier.clone(), }) } diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda/src/lib.rs index ea235403c11..923b526721a 100644 --- a/quickwit/quickwit-lambda/src/lib.rs +++ b/quickwit/quickwit-lambda/src/lib.rs @@ -17,15 +17,18 @@ //! This crate provides: //! - A Lambda handler that executes leaf search requests //! - An AWS Lambda implementation of the `RemoteFunctionInvoker` trait +//! - Auto-deployment functionality for Lambda functions mod config; mod context; +mod deployer; mod error; mod handler; mod invoker; pub use config::{LambdaConfig, LambdaSearcherConfig}; pub use context::LambdaSearcherContext; +pub use deployer::LambdaDeployer; pub use error::{LambdaError, LambdaResult}; pub use handler::{LeafSearchPayload, LeafSearchResponsePayload, handle_leaf_search}; pub use invoker::AwsLambdaInvoker; diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 4caf2587909..514277b2148 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -685,7 +685,8 @@ fn visit_aggregation_mut( modified_something } -// equivalent to Bound::map, which is unstable +/// Maps a `Bound` to a `Bound` by applying a function to the contained value. +/// Equivalent to `Bound::map`, which is currently unstable. pub fn map_bound(bound: Bound, f: impl FnOnce(T) -> U) -> Bound { use Bound::*; match bound { diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 6f12272a204..34449309e24 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -23,6 +23,8 @@ mod collector; mod error; mod fetch_docs; mod find_trace_ids_collector; + +/// Leaf search operations. pub mod leaf; mod leaf_cache; mod list_fields; diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index dd12ea3637d..b107e79ba07 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -87,7 +87,7 @@ use quickwit_ingest::{ }; use quickwit_jaeger::JaegerService; use quickwit_janitor::{JanitorService, start_janitor_service}; -use quickwit_lambda::AwsLambdaInvoker; +use quickwit_lambda::{AwsLambdaInvoker, LambdaDeployer}; use quickwit_metastore::{ ControlPlaneMetastore, ListIndexesMetadataResponseExt, MetastoreResolver, }; @@ -1023,6 +1023,19 @@ async fn setup_searcher( let lambda_invoker: Option> = if node_config.searcher_config.lambda.enabled { info!("initializing AWS Lambda invoker for leaf search"); + + // Auto-deploy Lambda function if configured + if node_config.searcher_config.lambda.auto_deploy { + info!("auto-deploying Lambda function"); + let deployer = LambdaDeployer::new() + .await + .context("failed to create Lambda deployer")?; + deployer + .deploy(&node_config.searcher_config.lambda) + .await + .context("failed to deploy Lambda function")?; + } + let invoker = AwsLambdaInvoker::new(&node_config.searcher_config.lambda) .await .context("failed to initialize AWS Lambda invoker")?; From 6c5fbf40b681685ddca723b16343a067426e43cb Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 16:03:50 +0100 Subject: [PATCH 08/22] rationalizing code --- .../quickwit-config/src/node_config/mod.rs | 7 - quickwit/quickwit-lambda/README.md | 5 +- quickwit/quickwit-lambda/build.rs | 156 +++++++ .../quickwit-lambda/src/bin/leaf_search.rs | 19 +- quickwit/quickwit-lambda/src/config.rs | 7 + quickwit/quickwit-lambda/src/context.rs | 34 +- quickwit/quickwit-lambda/src/deployer.rs | 420 ++++++++++++++++++ quickwit/quickwit-lambda/src/handler.rs | 3 +- 8 files changed, 602 insertions(+), 49 deletions(-) create mode 100644 quickwit/quickwit-lambda/build.rs create mode 100644 quickwit/quickwit-lambda/src/deployer.rs diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index 238e237430e..fc9914be9ab 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -336,9 +336,6 @@ pub struct LambdaConfig { /// Memory size for the Lambda function in MB (default: 1024). #[serde(default = "LambdaConfig::default_memory_size_mb")] pub memory_size_mb: u32, - /// Timeout for the Lambda function in seconds (default: 30). - #[serde(default = "LambdaConfig::default_timeout_secs")] - pub timeout_secs: u32, } impl Default for LambdaConfig { @@ -353,7 +350,6 @@ impl Default for LambdaConfig { auto_deploy: false, execution_role_arn: None, memory_size_mb: Self::default_memory_size_mb(), - timeout_secs: Self::default_timeout_secs(), } } } @@ -374,9 +370,6 @@ impl LambdaConfig { fn default_memory_size_mb() -> u32 { 1024 } - fn default_timeout_secs() -> u32 { - 30 - } } #[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] diff --git a/quickwit/quickwit-lambda/README.md b/quickwit/quickwit-lambda/README.md index 5734c6cc55d..48644e5f29f 100644 --- a/quickwit/quickwit-lambda/README.md +++ b/quickwit/quickwit-lambda/README.md @@ -41,12 +41,11 @@ searcher: | `function_name` | string | `"quickwit-lambda-search"` | Lambda function name or ARN | | `function_qualifier` | string | `null` | Optional alias or version qualifier | | `max_splits_per_invocation` | int | `10` | Maximum splits per Lambda invocation | -| `invocation_timeout_secs` | int | `30` | Client-side timeout for invocations | +| `invocation_timeout_secs` | int | `30` | Timeout in seconds (used for both client-side and Lambda function timeout when auto-deploying) | | `max_concurrent_invocations` | int | `100` | Maximum concurrent Lambda invocations | | `auto_deploy` | bool | `false` | Enable automatic Lambda deployment | | `execution_role_arn` | string | `null` | IAM role ARN (required for auto-deploy) | | `memory_size_mb` | int | `1024` | Lambda memory allocation in MB | -| `timeout_secs` | int | `30` | Lambda function timeout in seconds | ## Auto-Deploy Feature @@ -211,7 +210,7 @@ If Quickwit fails to start with Lambda enabled: ### Runtime Issues -1. **Timeouts**: Increase `invocation_timeout_secs` and/or `timeout_secs` +1. **Timeouts**: Increase `invocation_timeout_secs` 2. **Memory errors**: Increase `memory_size_mb` 3. **Throttling**: AWS Lambda has concurrency limits - check your account limits diff --git a/quickwit/quickwit-lambda/build.rs b/quickwit/quickwit-lambda/build.rs new file mode 100644 index 00000000000..12406e52ce5 --- /dev/null +++ b/quickwit/quickwit-lambda/build.rs @@ -0,0 +1,156 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Build script for quickwit-lambda. +//! +//! When the `auto-deploy` feature is enabled, this script: +//! 1. Looks for a pre-built Lambda binary at a known location +//! 2. Creates a zip file containing the binary named "bootstrap" +//! 3. Places the zip in OUT_DIR for embedding via include_bytes! +//! +//! The Lambda binary should be pre-built in CI for the aarch64-unknown-linux-musl +//! target and placed in the expected location. + +fn main() { + #[cfg(feature = "auto-deploy")] + auto_deploy_build(); + + #[cfg(not(feature = "auto-deploy"))] + println!("cargo:rerun-if-changed=build.rs"); +} + +#[cfg(feature = "auto-deploy")] +fn auto_deploy_build() { + use std::env; + use std::path::PathBuf; + + println!("cargo:rerun-if-changed=build.rs"); + println!("cargo:rerun-if-env-changed=QUICKWIT_LAMBDA_BINARY_PATH"); + + let out_dir = PathBuf::from(env::var("OUT_DIR").expect("OUT_DIR not set")); + let zip_path = out_dir.join("lambda_bootstrap.zip"); + + // Look for the pre-built Lambda binary in order of preference: + // 1. Environment variable QUICKWIT_LAMBDA_BINARY_PATH + // 2. Target directory (for local builds with cross-compilation) + // 3. Fallback to creating a placeholder for development + + let binary_path = if let Ok(path) = env::var("QUICKWIT_LAMBDA_BINARY_PATH") { + println!( + "cargo:warning=Using Lambda binary from QUICKWIT_LAMBDA_BINARY_PATH: {}", + path + ); + Some(PathBuf::from(path)) + } else { + // Try to find in target directory + let workspace_root = PathBuf::from(env::var("CARGO_MANIFEST_DIR").unwrap()) + .parent() + .unwrap() + .parent() + .unwrap() + .to_path_buf(); + + let potential_paths = [ + workspace_root + .join("target/aarch64-unknown-linux-musl/release/quickwit-lambda-leaf-search"), + workspace_root.join("target/lambda/quickwit-lambda-leaf-search/bootstrap"), + workspace_root.join("lambda_bootstrap"), + ]; + + potential_paths.into_iter().find(|p| p.exists()) + }; + + match binary_path { + Some(path) => { + println!("cargo:warning=Packaging Lambda binary from: {:?}", path); + create_lambda_zip(&path, &zip_path); + } + None => { + // Create a placeholder zip for development builds + // This allows compilation to succeed, but deploy() will fail at runtime + // if someone tries to use auto-deploy without a proper binary + println!("cargo:warning=No Lambda binary found, creating placeholder zip"); + println!("cargo:warning=Set QUICKWIT_LAMBDA_BINARY_PATH or build the binary first"); + create_placeholder_zip(&zip_path); + } + } +} + +#[cfg(feature = "auto-deploy")] +fn create_lambda_zip(binary_path: &std::path::Path, zip_path: &std::path::Path) { + use std::fs::File; + use std::io::Read; + + use zip::write::FileOptions; + use zip::ZipWriter; + + let mut binary_data = Vec::new(); + File::open(binary_path) + .expect("Failed to open Lambda binary") + .read_to_end(&mut binary_data) + .expect("Failed to read Lambda binary"); + + let zip_file = File::create(zip_path).expect("Failed to create zip file"); + let mut zip = ZipWriter::new(zip_file); + + // Lambda requires the binary to be named "bootstrap" with executable permissions + let options = FileOptions::default() + .compression_method(zip::CompressionMethod::Deflated) + .unix_permissions(0o755); + + zip.start_file("bootstrap", options) + .expect("Failed to start zip file entry"); + + std::io::Write::write_all(&mut zip, &binary_data).expect("Failed to write binary to zip"); + + zip.finish().expect("Failed to finalize zip file"); + + println!( + "cargo:warning=Created Lambda zip at {:?} ({} bytes)", + zip_path, + std::fs::metadata(zip_path).unwrap().len() + ); +} + +#[cfg(feature = "auto-deploy")] +fn create_placeholder_zip(zip_path: &std::path::Path) { + use std::fs::File; + + use zip::write::FileOptions; + use zip::ZipWriter; + + let zip_file = File::create(zip_path).expect("Failed to create placeholder zip file"); + let mut zip = ZipWriter::new(zip_file); + + // Create a placeholder script that returns an error + let placeholder_script = r#"#!/bin/sh +echo "ERROR: This is a placeholder Lambda binary." +echo "The auto-deploy feature requires a properly built Lambda binary." +echo "Please build the quickwit-lambda-leaf-search binary for aarch64-unknown-linux-musl" +echo "and set QUICKWIT_LAMBDA_BINARY_PATH environment variable." +exit 1 +"#; + + let options = FileOptions::default() + .compression_method(zip::CompressionMethod::Deflated) + .unix_permissions(0o755); + + zip.start_file("bootstrap", options) + .expect("Failed to start zip file entry"); + + std::io::Write::write_all(&mut zip, placeholder_script.as_bytes()) + .expect("Failed to write placeholder to zip"); + + zip.finish().expect("Failed to finalize zip file"); +} diff --git a/quickwit/quickwit-lambda/src/bin/leaf_search.rs b/quickwit/quickwit-lambda/src/bin/leaf_search.rs index 4f8412534fb..376e23dbe71 100644 --- a/quickwit/quickwit-lambda/src/bin/leaf_search.rs +++ b/quickwit/quickwit-lambda/src/bin/leaf_search.rs @@ -14,6 +14,8 @@ //! AWS Lambda binary entry point for Quickwit leaf search. +use std::sync::Arc; + use lambda_runtime::{Error, LambdaEvent, service_fn}; use quickwit_lambda::{LambdaSearcherContext, LeafSearchPayload, handle_leaf_search}; use tracing::info; @@ -27,19 +29,20 @@ async fn main() -> Result<(), Error> { .json() .init(); - // Pre-initialize context on cold start - let context = LambdaSearcherContext::get_or_init().await; + // Initialize context on cold start (wrapped in Arc for sharing across invocations) + let context = Arc::new(LambdaSearcherContext::try_from_env()?); info!("lambda context initialized, starting handler loop"); // Run the Lambda handler - lambda_runtime::run(service_fn( - |event: LambdaEvent| async move { - let (payload, _lambda_ctx) = event.into_parts(); - handle_leaf_search(payload, context) + lambda_runtime::run(service_fn(|event: LambdaEvent| { + let ctx = Arc::clone(&context); + async move { + let (payload, _event_ctx) = event.into_parts(); + handle_leaf_search(payload, &ctx) .await .map_err(|e| lambda_runtime::Error::from(e.to_string())) - }, - )) + } + })) .await } diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda/src/config.rs index 6178256ddfd..6a758952995 100644 --- a/quickwit/quickwit-lambda/src/config.rs +++ b/quickwit/quickwit-lambda/src/config.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::Context as _; use bytesize::ByteSize; // Re-export LambdaConfig from quickwit-config pub use quickwit_config::LambdaConfig; @@ -28,6 +29,12 @@ pub struct LambdaSearcherConfig { } impl LambdaSearcherConfig { + + pub fn try_from_env() -> anyhow::Result { + let memory_mb: usize = quickwit_common::get_from_env_opt("AWS_LAMBDA_FUNCTION_MEMORY_SIZE", false) + .context("could not get aws lambda function memory size from ENV")?; + Ok(LambdaSearcherConfig::for_memory(memory_mb)) + } /// Create a Lambda-optimized searcher config based on the allocated memory. pub fn for_memory(memory_mb: usize) -> Self { // Warmup budget is about half of memory diff --git a/quickwit/quickwit-lambda/src/context.rs b/quickwit/quickwit-lambda/src/context.rs index 3bc00c17856..cd660d438d7 100644 --- a/quickwit/quickwit-lambda/src/context.rs +++ b/quickwit/quickwit-lambda/src/context.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::{Arc, OnceLock}; +use std::sync::Arc; use quickwit_config::SearcherConfig; use quickwit_search::SearcherContext; @@ -20,18 +20,19 @@ use quickwit_storage::StorageResolver; use tracing::info; use crate::config::LambdaSearcherConfig; -use crate::error::LambdaResult; -/// Lambda-specific searcher context that caches resources across warm invocations. +/// Lambda-specific searcher context that holds resources for search execution. pub struct LambdaSearcherContext { pub searcher_context: Arc, pub storage_resolver: StorageResolver, } impl LambdaSearcherContext { - pub async fn new(config: LambdaSearcherConfig) -> LambdaResult { + /// Create a new Lambda searcher context from environment variables. + pub fn try_from_env() -> anyhow::Result { info!("Initializing Lambda searcher context"); + let config = LambdaSearcherConfig::try_from_env()?; let searcher_config = create_searcher_config(&config); let searcher_context = Arc::new(SearcherContext::new(searcher_config, None)); let storage_resolver = StorageResolver::configured(&Default::default()); @@ -41,22 +42,6 @@ impl LambdaSearcherContext { storage_resolver, }) } - - pub async fn get_or_init() -> &'static Self { - static CONTEXT: OnceLock = OnceLock::new(); - - if let Some(ctx) = CONTEXT.get() { - return ctx; - } - - let config = config_from_env(); - let ctx = Self::new(config) - .await - .expect("Failed to initialize Lambda searcher context"); - - let _ = CONTEXT.set(ctx); - CONTEXT.get().unwrap() - } } fn create_searcher_config(config: &LambdaSearcherConfig) -> SearcherConfig { @@ -64,12 +49,3 @@ fn create_searcher_config(config: &LambdaSearcherConfig) -> SearcherConfig { searcher_config.max_num_concurrent_split_searches = config.max_concurrent_split_searches; searcher_config } - -fn config_from_env() -> LambdaSearcherConfig { - let memory_mb = std::env::var("AWS_LAMBDA_FUNCTION_MEMORY_SIZE") - .ok() - .and_then(|s| s.parse().ok()) - .unwrap_or(1024); - - LambdaSearcherConfig::for_memory(memory_mb) -} diff --git a/quickwit/quickwit-lambda/src/deployer.rs b/quickwit/quickwit-lambda/src/deployer.rs new file mode 100644 index 00000000000..9c2a937a57b --- /dev/null +++ b/quickwit/quickwit-lambda/src/deployer.rs @@ -0,0 +1,420 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Lambda function deployment for auto-deploy feature. +//! +//! This module provides functionality to automatically deploy or update +//! the Lambda function used for leaf search operations. + +#[cfg(feature = "auto-deploy")] +use std::collections::HashMap; + +use aws_sdk_lambda::Client as LambdaClient; +#[cfg(feature = "auto-deploy")] +use aws_sdk_lambda::error::SdkError; +#[cfg(feature = "auto-deploy")] +use aws_sdk_lambda::operation::create_function::CreateFunctionError; +#[cfg(feature = "auto-deploy")] +use aws_sdk_lambda::operation::get_function::GetFunctionOutput; +#[cfg(feature = "auto-deploy")] +use aws_sdk_lambda::primitives::Blob; +#[cfg(feature = "auto-deploy")] +use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; +#[cfg(feature = "auto-deploy")] +use tracing::{debug, info, warn}; + +use crate::config::LambdaConfig; +use crate::error::{LambdaError, LambdaResult}; + +/// Embedded Lambda binary (arm64, compressed). +/// This is included at compile time when the `auto-deploy` feature is enabled. +#[cfg(feature = "auto-deploy")] +const LAMBDA_BINARY: &[u8] = include_bytes!(concat!(env!("OUT_DIR"), "/lambda_bootstrap.zip")); + +/// Version tag key used to track deployed Quickwit version. +#[cfg(feature = "auto-deploy")] +const VERSION_TAG_KEY: &str = "quickwit_version"; + +/// Description prefix for auto-deployed Lambda functions. +#[cfg(feature = "auto-deploy")] +const FUNCTION_DESCRIPTION: &str = "Quickwit Lambda leaf search handler"; + +/// Lambda function deployer. +/// +/// Handles creating and updating Lambda functions for the auto-deploy feature. +/// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. +pub struct LambdaDeployer { + #[allow(dead_code)] + client: LambdaClient, +} + +impl LambdaDeployer { + /// Create a new Lambda deployer using default AWS configuration. + pub async fn new() -> LambdaResult { + let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; + let client = LambdaClient::new(&aws_config); + Ok(Self { client }) + } + + /// Create a new Lambda deployer with a custom client. + #[allow(dead_code)] + pub fn with_client(client: LambdaClient) -> Self { + Self { client } + } + + /// Deploy or update the Lambda function. + /// + /// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. + /// Returns the function ARN. + #[cfg(feature = "auto-deploy")] + pub async fn deploy(&self, config: &LambdaConfig) -> LambdaResult { + let function_name = &config.function_name; + let role_arn = config.execution_role_arn.as_ref().ok_or_else(|| { + LambdaError::Configuration("execution_role_arn required for auto_deploy".into()) + })?; + + match self.get_function(function_name).await { + Ok(existing) => { + self.update_function_if_needed(function_name, &existing, config) + .await + } + Err(LambdaError::NotFound(_)) => { + // Function doesn't exist, try to create it + match self.create_function(function_name, role_arn, config).await { + Ok(arn) => Ok(arn), + Err(LambdaError::ResourceConflict) => { + // Another node created the function concurrently, update instead + info!( + function_name = %function_name, + "function was created concurrently by another node, updating instead" + ); + let existing = self.get_function(function_name).await?; + self.update_function_if_needed(function_name, &existing, config) + .await + } + Err(e) => Err(e), + } + } + Err(e) => Err(e), + } + } + + /// Deploy is a no-op when auto-deploy feature is not enabled. + #[cfg(not(feature = "auto-deploy"))] + pub async fn deploy(&self, _config: &LambdaConfig) -> LambdaResult { + Err(LambdaError::Configuration( + "auto-deploy feature is not enabled at compile time".into(), + )) + } + + /// Create the Lambda function. + /// + /// Note: CreateFunction is idempotent - if the function already exists, AWS returns + /// ResourceConflictException. Multiple Quickwit nodes starting simultaneously is safe; + /// one will succeed and others will fall back to update_function. + #[cfg(feature = "auto-deploy")] + async fn create_function( + &self, + name: &str, + role: &str, + config: &LambdaConfig, + ) -> LambdaResult { + info!( + function_name = %name, + role = %role, + memory_mb = config.memory_size_mb, + timeout_secs = config.invocation_timeout_secs, + "creating Lambda function" + ); + + let environment = self.build_environment(); + let tags = self.build_tags(); + + let result = self + .client + .create_function() + .function_name(name) + .runtime(Runtime::Providedal2023) + .architectures(Architecture::Arm64) + .handler("bootstrap") + .role(role) + .code( + FunctionCode::builder() + .zip_file(Blob::new(LAMBDA_BINARY)) + .build(), + ) + .memory_size(config.memory_size_mb as i32) + .timeout(config.invocation_timeout_secs as i32) + .environment(environment) + .description(format!( + "{} (v{})", + FUNCTION_DESCRIPTION, + env!("CARGO_PKG_VERSION") + )) + .set_tags(Some(tags)) + .send() + .await; + + match result { + Ok(output) => { + let arn = output + .function_arn() + .ok_or_else(|| LambdaError::Deployment("no function ARN returned".into()))? + .to_string(); + info!(function_arn = %arn, "Lambda function created successfully"); + Ok(arn) + } + Err(SdkError::ServiceError(err)) + if matches!(err.err(), CreateFunctionError::ResourceConflictException(_)) => + { + Err(LambdaError::ResourceConflict) + } + Err(e) => Err(LambdaError::Deployment(format!( + "failed to create function: {}", + e + ))), + } + } + + /// Update the Lambda function if needed. + /// + /// Compares the deployed version tag with the current Quickwit version + /// and updates if they differ. + #[cfg(feature = "auto-deploy")] + async fn update_function_if_needed( + &self, + name: &str, + existing: &GetFunctionOutput, + config: &LambdaConfig, + ) -> LambdaResult { + let function_arn = existing + .configuration() + .and_then(|c| c.function_arn()) + .ok_or_else(|| LambdaError::Deployment("no function ARN in existing config".into()))? + .to_string(); + + if !self.needs_update(existing) { + debug!( + function_name = %name, + "Lambda function is up to date, skipping update" + ); + return Ok(function_arn); + } + + info!( + function_name = %name, + "updating Lambda function to version {}", + env!("CARGO_PKG_VERSION") + ); + + // Update function code + self.client + .update_function_code() + .function_name(name) + .zip_file(Blob::new(LAMBDA_BINARY)) + .architectures(Architecture::Arm64) + .send() + .await + .map_err(|e| { + LambdaError::Deployment(format!("failed to update function code: {}", e)) + })?; + + // Wait for the update to complete before updating configuration + self.wait_for_update_complete(name).await?; + + // Update function configuration + self.client + .update_function_configuration() + .function_name(name) + .memory_size(config.memory_size_mb as i32) + .timeout(config.invocation_timeout_secs as i32) + .environment(self.build_environment()) + .description(format!( + "{} (v{})", + FUNCTION_DESCRIPTION, + env!("CARGO_PKG_VERSION") + )) + .send() + .await + .map_err(|e| { + LambdaError::Deployment(format!("failed to update function configuration: {}", e)) + })?; + + // Wait for config update to complete before updating tags + self.wait_for_update_complete(name).await?; + + // Update tags + self.client + .tag_resource() + .resource(function_arn.clone()) + .set_tags(Some(self.build_tags())) + .send() + .await + .map_err(|e| { + LambdaError::Deployment(format!("failed to update function tags: {}", e)) + })?; + + info!(function_arn = %function_arn, "Lambda function updated successfully"); + Ok(function_arn) + } + + /// Wait for function update to complete. + #[cfg(feature = "auto-deploy")] + async fn wait_for_update_complete(&self, name: &str) -> LambdaResult<()> { + // Poll until the function state is Active and LastUpdateStatus is Successful + for _ in 0..60 { + let output = self.get_function(name).await?; + if let Some(config) = output.configuration() { + let state = config.state(); + let last_update_status = config.last_update_status(); + + use aws_sdk_lambda::types::{LastUpdateStatus, State}; + match (state, last_update_status) { + (Some(State::Active), Some(LastUpdateStatus::Successful)) => { + return Ok(()); + } + (Some(State::Failed), _) | (_, Some(LastUpdateStatus::Failed)) => { + let reason = config + .last_update_status_reason() + .unwrap_or("unknown reason"); + return Err(LambdaError::Deployment(format!( + "function update failed: {}", + reason + ))); + } + _ => { + debug!( + function_name = %name, + state = ?state, + last_update_status = ?last_update_status, + "waiting for function update to complete" + ); + tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; + } + } + } + } + Err(LambdaError::Deployment( + "timeout waiting for function update to complete".into(), + )) + } + + /// Check if the function needs to be updated based on version tag. + #[cfg(feature = "auto-deploy")] + fn needs_update(&self, existing: &GetFunctionOutput) -> bool { + let current_version = env!("CARGO_PKG_VERSION"); + + let deployed_version = existing.tags().and_then(|t| t.get(VERSION_TAG_KEY)); + + match deployed_version { + Some(version) if version == current_version => { + debug!( + deployed_version = %version, + current_version = %current_version, + "versions match, no update needed" + ); + false + } + Some(version) => { + info!( + deployed_version = %version, + current_version = %current_version, + "version mismatch, update needed" + ); + true + } + None => { + warn!("no version tag found on deployed function, update needed"); + true + } + } + } + + /// Get function details from AWS. + #[cfg(feature = "auto-deploy")] + async fn get_function(&self, name: &str) -> LambdaResult { + self.client + .get_function() + .function_name(name) + .send() + .await + .map_err(|e| { + // Check if it's a not found error + if e.to_string().contains("ResourceNotFoundException") + || e.to_string().contains("Function not found") + { + LambdaError::NotFound(name.to_string()) + } else { + LambdaError::Deployment(format!("failed to get function: {}", e)) + } + }) + } + + /// Build environment variables for the Lambda function. + #[cfg(feature = "auto-deploy")] + fn build_environment(&self) -> Environment { + let mut env_vars = HashMap::new(); + // Set reasonable defaults for logging + env_vars.insert("RUST_LOG".to_string(), "info".to_string()); + env_vars.insert("RUST_BACKTRACE".to_string(), "1".to_string()); + + Environment::builder().set_variables(Some(env_vars)).build() + } + + /// Build tags for the Lambda function. + #[cfg(feature = "auto-deploy")] + fn build_tags(&self) -> HashMap { + let mut tags = HashMap::new(); + tags.insert( + VERSION_TAG_KEY.to_string(), + env!("CARGO_PKG_VERSION").to_string(), + ); + tags.insert("managed_by".to_string(), "quickwit".to_string()); + tags + } +} + +#[cfg(all(test, feature = "auto-deploy"))] +mod tests { + use super::*; + + #[test] + fn test_build_tags() { + let deployer = LambdaDeployer { + client: LambdaClient::from_conf( + aws_sdk_lambda::Config::builder() + .behavior_version(aws_sdk_lambda::config::BehaviorVersion::latest()) + .build(), + ), + }; + let tags = deployer.build_tags(); + assert!(tags.contains_key(VERSION_TAG_KEY)); + assert_eq!(tags.get("managed_by"), Some(&"quickwit".to_string())); + } + + #[test] + fn test_build_environment() { + let deployer = LambdaDeployer { + client: LambdaClient::from_conf( + aws_sdk_lambda::Config::builder() + .behavior_version(aws_sdk_lambda::config::BehaviorVersion::latest()) + .build(), + ), + }; + let env = deployer.build_environment(); + let vars = env.variables().unwrap(); + assert_eq!(vars.get("RUST_LOG"), Some(&"info".to_string())); + assert_eq!(vars.get("RUST_BACKTRACE"), Some(&"1".to_string())); + } +} diff --git a/quickwit/quickwit-lambda/src/handler.rs b/quickwit/quickwit-lambda/src/handler.rs index 41399021a9a..24231916615 100644 --- a/quickwit/quickwit-lambda/src/handler.rs +++ b/quickwit/quickwit-lambda/src/handler.rs @@ -64,8 +64,7 @@ pub async fn handle_leaf_search( leaf_search_request, &ctx.storage_resolver, ) - .await - .map_err(LambdaError::from)?; + .await?; info!( num_hits = leaf_search_response.num_hits, From 494a14432c73fddcb9a94d0995642de221f768f8 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 16:22:59 +0100 Subject: [PATCH 09/22] blop --- quickwit/quickwit-config/src/node_config/mod.rs | 7 ------- quickwit/quickwit-lambda/README.md | 1 - 2 files changed, 8 deletions(-) diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index fc9914be9ab..04033dd94e0 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -324,9 +324,6 @@ pub struct LambdaConfig { /// Timeout for Lambda invocations in seconds. #[serde(default = "LambdaConfig::default_invocation_timeout_secs")] pub invocation_timeout_secs: u64, - /// Maximum number of concurrent Lambda invocations. - #[serde(default = "LambdaConfig::default_max_concurrent_invocations")] - pub max_concurrent_invocations: usize, /// Enable automatic Lambda function deployment at startup. #[serde(default)] pub auto_deploy: bool, @@ -346,7 +343,6 @@ impl Default for LambdaConfig { function_qualifier: None, max_splits_per_invocation: Self::default_max_splits_per_invocation(), invocation_timeout_secs: Self::default_invocation_timeout_secs(), - max_concurrent_invocations: Self::default_max_concurrent_invocations(), auto_deploy: false, execution_role_arn: None, memory_size_mb: Self::default_memory_size_mb(), @@ -364,9 +360,6 @@ impl LambdaConfig { fn default_invocation_timeout_secs() -> u64 { 30 } - fn default_max_concurrent_invocations() -> usize { - 100 - } fn default_memory_size_mb() -> u32 { 1024 } diff --git a/quickwit/quickwit-lambda/README.md b/quickwit/quickwit-lambda/README.md index 48644e5f29f..1d9516e177c 100644 --- a/quickwit/quickwit-lambda/README.md +++ b/quickwit/quickwit-lambda/README.md @@ -42,7 +42,6 @@ searcher: | `function_qualifier` | string | `null` | Optional alias or version qualifier | | `max_splits_per_invocation` | int | `10` | Maximum splits per Lambda invocation | | `invocation_timeout_secs` | int | `30` | Timeout in seconds (used for both client-side and Lambda function timeout when auto-deploying) | -| `max_concurrent_invocations` | int | `100` | Maximum concurrent Lambda invocations | | `auto_deploy` | bool | `false` | Enable automatic Lambda deployment | | `execution_role_arn` | string | `null` | IAM role ARN (required for auto-deploy) | | `memory_size_mb` | int | `1024` | Lambda memory allocation in MB | From 3ac52cd00c965973aa70f148c43e34844b762886 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 16:43:27 +0100 Subject: [PATCH 10/22] better config --- quickwit/quickwit-config/src/lib.rs | 4 +- .../quickwit-config/src/node_config/mod.rs | 65 +++++++++++-------- quickwit/quickwit-lambda/README.md | 31 ++++++--- quickwit/quickwit-lambda/src/config.rs | 4 +- quickwit/quickwit-lambda/src/deployer.rs | 32 +++++---- quickwit/quickwit-search/src/root.rs | 3 +- quickwit/quickwit-serve/src/lib.rs | 8 +-- 7 files changed, 90 insertions(+), 57 deletions(-) diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index 59b019e8f0b..22cdb2538b4 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -74,8 +74,8 @@ pub use crate::metastore_config::{ }; pub use crate::node_config::{ CacheConfig, CachePolicy, DEFAULT_QW_CONFIG_PATH, GrpcConfig, IndexerConfig, IngestApiConfig, - JaegerConfig, KeepAliveConfig, LambdaConfig, NodeConfig, RestConfig, SearcherConfig, - SplitCacheLimits, StorageTimeoutPolicy, TlsConfig, + JaegerConfig, KeepAliveConfig, LambdaConfig, LambdaDeployConfig, NodeConfig, RestConfig, + SearcherConfig, SplitCacheLimits, StorageTimeoutPolicy, TlsConfig, }; use crate::source_config::serialize::{SourceConfigV0_7, SourceConfigV0_8, VersionedSourceConfig}; pub use crate::storage_config::{ diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index 04033dd94e0..adef2e11651 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -301,17 +301,15 @@ pub struct SearcherConfig { pub warmup_memory_budget: ByteSize, pub warmup_single_split_initial_allocation: ByteSize, /// Lambda configuration for serverless leaf search execution. + /// If set, enables Lambda execution for leaf search. #[serde(default)] - pub lambda: LambdaConfig, + pub lambda: Option, } /// Configuration for AWS Lambda leaf search execution. #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] #[serde(deny_unknown_fields, default)] pub struct LambdaConfig { - /// Enable Lambda execution mode for leaf search. - #[serde(default)] - pub enabled: bool, /// AWS Lambda function name or ARN. #[serde(default = "LambdaConfig::default_function_name")] pub function_name: String, @@ -321,31 +319,52 @@ pub struct LambdaConfig { /// Maximum number of splits per Lambda invocation. #[serde(default = "LambdaConfig::default_max_splits_per_invocation")] pub max_splits_per_invocation: usize, - /// Timeout for Lambda invocations in seconds. - #[serde(default = "LambdaConfig::default_invocation_timeout_secs")] - pub invocation_timeout_secs: u64, - /// Enable automatic Lambda function deployment at startup. + /// Auto-deploy configuration. If set, Quickwit will automatically deploy + /// the Lambda function at startup. #[serde(default)] - pub auto_deploy: bool, - /// IAM execution role ARN for the Lambda function (required if auto_deploy=true). - #[serde(default)] - pub execution_role_arn: Option, - /// Memory size for the Lambda function in MB (default: 1024). - #[serde(default = "LambdaConfig::default_memory_size_mb")] + pub auto_deploy: Option, +} + +/// Configuration for automatic Lambda function deployment. +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(deny_unknown_fields, default)] +pub struct LambdaDeployConfig { + /// IAM execution role ARN for the Lambda function. + pub execution_role_arn: String, + /// Memory size for the Lambda function in MB. + #[serde(default = "LambdaDeployConfig::default_memory_size_mb")] pub memory_size_mb: u32, + /// Timeout for Lambda invocations in seconds. + #[serde(default = "LambdaDeployConfig::default_invocation_timeout_secs")] + pub invocation_timeout_secs: u64, +} + +impl Default for LambdaDeployConfig { + fn default() -> Self { + Self { + execution_role_arn: String::new(), + memory_size_mb: Self::default_memory_size_mb(), + invocation_timeout_secs: Self::default_invocation_timeout_secs(), + } + } +} + +impl LambdaDeployConfig { + fn default_memory_size_mb() -> u32 { + 1024 + } + fn default_invocation_timeout_secs() -> u64 { + 30 + } } impl Default for LambdaConfig { fn default() -> Self { Self { - enabled: false, function_name: Self::default_function_name(), function_qualifier: None, max_splits_per_invocation: Self::default_max_splits_per_invocation(), - invocation_timeout_secs: Self::default_invocation_timeout_secs(), - auto_deploy: false, - execution_role_arn: None, - memory_size_mb: Self::default_memory_size_mb(), + auto_deploy: None, } } } @@ -357,12 +376,6 @@ impl LambdaConfig { fn default_max_splits_per_invocation() -> usize { 10 } - fn default_invocation_timeout_secs() -> u64 { - 30 - } - fn default_memory_size_mb() -> u32 { - 1024 - } } #[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] @@ -498,7 +511,7 @@ impl Default for SearcherConfig { storage_timeout_policy: None, warmup_memory_budget: ByteSize::gb(100), warmup_single_split_initial_allocation: ByteSize::gb(1), - lambda: LambdaConfig::default(), + lambda: None, } } } diff --git a/quickwit/quickwit-lambda/README.md b/quickwit/quickwit-lambda/README.md index 1d9516e177c..842a0c3ef7d 100644 --- a/quickwit/quickwit-lambda/README.md +++ b/quickwit/quickwit-lambda/README.md @@ -22,29 +22,40 @@ When the `auto-deploy` feature is enabled, Quickwit can automatically create or ## Configuration -Add the following to your Quickwit configuration: +Add the following to your Quickwit configuration. Lambda is enabled by including the `lambda` section under `searcher`: ```yaml searcher: lambda: - enabled: true + # Lambda function settings + function_name: "quickwit-lambda-search" + max_splits_per_invocation: 10 + # Auto-deploy settings (requires auto-deploy feature) - auto_deploy: true - execution_role_arn: "arn:aws:iam::123456789:role/quickwit-lambda-role" + auto_deploy: + execution_role_arn: "arn:aws:iam::123456789:role/quickwit-lambda-role" + memory_size_mb: 1024 + invocation_timeout_secs: 30 ``` ### Configuration Options +#### LambdaConfig (under `searcher.lambda`) + | Option | Type | Default | Description | |--------|------|---------|-------------| -| `enabled` | bool | `false` | Enable Lambda execution for leaf search | | `function_name` | string | `"quickwit-lambda-search"` | Lambda function name or ARN | | `function_qualifier` | string | `null` | Optional alias or version qualifier | | `max_splits_per_invocation` | int | `10` | Maximum splits per Lambda invocation | -| `invocation_timeout_secs` | int | `30` | Timeout in seconds (used for both client-side and Lambda function timeout when auto-deploying) | -| `auto_deploy` | bool | `false` | Enable automatic Lambda deployment | -| `execution_role_arn` | string | `null` | IAM role ARN (required for auto-deploy) | +| `auto_deploy` | object | `null` | Auto-deploy configuration (see below) | + +#### LambdaDeployConfig (under `searcher.lambda.auto_deploy`) + +| Option | Type | Default | Description | +|--------|------|---------|-------------| +| `execution_role_arn` | string | required | IAM role ARN for the Lambda function | | `memory_size_mb` | int | `1024` | Lambda memory allocation in MB | +| `invocation_timeout_secs` | int | `30` | Timeout for Lambda invocations in seconds | ## Auto-Deploy Feature @@ -82,7 +93,7 @@ cargo build --release --features auto-deploy ### Startup Behavior -When `auto_deploy: true` is configured, the following happens at Quickwit startup: +When `auto_deploy` is configured, the following happens at Quickwit startup: 1. **Function Check**: Quickwit checks if the Lambda function exists 2. **Create or Update**: @@ -203,7 +214,7 @@ The Lambda function: If Quickwit fails to start with Lambda enabled: -1. **"execution_role_arn required for auto_deploy"**: You enabled `auto_deploy` but didn't provide an IAM role ARN +1. **"execution_role_arn required"**: The `auto_deploy` section is present but the `execution_role_arn` field is empty 2. **"failed to create function"**: Check IAM permissions for the Quickwit process 3. **"Lambda function validation failed"**: The function exists but isn't invocable - check the function's execution role diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda/src/config.rs index 6a758952995..dc69dea727a 100644 --- a/quickwit/quickwit-lambda/src/config.rs +++ b/quickwit/quickwit-lambda/src/config.rs @@ -14,8 +14,8 @@ use anyhow::Context as _; use bytesize::ByteSize; -// Re-export LambdaConfig from quickwit-config -pub use quickwit_config::LambdaConfig; +// Re-export LambdaConfig and LambdaDeployConfig from quickwit-config +pub use quickwit_config::{LambdaConfig, LambdaDeployConfig}; /// Configuration for the Lambda handler's SearcherContext. /// These settings are optimized for Lambda's memory constraints. diff --git a/quickwit/quickwit-lambda/src/deployer.rs b/quickwit/quickwit-lambda/src/deployer.rs index 9c2a937a57b..fcec6a4c48f 100644 --- a/quickwit/quickwit-lambda/src/deployer.rs +++ b/quickwit/quickwit-lambda/src/deployer.rs @@ -34,7 +34,7 @@ use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; #[cfg(feature = "auto-deploy")] use tracing::{debug, info, warn}; -use crate::config::LambdaConfig; +use crate::config::LambdaDeployConfig; use crate::error::{LambdaError, LambdaResult}; /// Embedded Lambda binary (arm64, compressed). @@ -78,20 +78,24 @@ impl LambdaDeployer { /// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. /// Returns the function ARN. #[cfg(feature = "auto-deploy")] - pub async fn deploy(&self, config: &LambdaConfig) -> LambdaResult { - let function_name = &config.function_name; - let role_arn = config.execution_role_arn.as_ref().ok_or_else(|| { - LambdaError::Configuration("execution_role_arn required for auto_deploy".into()) - })?; + pub async fn deploy( + &self, + function_name: &str, + deploy_config: &LambdaDeployConfig, + ) -> LambdaResult { + let role_arn = &deploy_config.execution_role_arn; match self.get_function(function_name).await { Ok(existing) => { - self.update_function_if_needed(function_name, &existing, config) + self.update_function_if_needed(function_name, &existing, deploy_config) .await } Err(LambdaError::NotFound(_)) => { // Function doesn't exist, try to create it - match self.create_function(function_name, role_arn, config).await { + match self + .create_function(function_name, role_arn, deploy_config) + .await + { Ok(arn) => Ok(arn), Err(LambdaError::ResourceConflict) => { // Another node created the function concurrently, update instead @@ -100,7 +104,7 @@ impl LambdaDeployer { "function was created concurrently by another node, updating instead" ); let existing = self.get_function(function_name).await?; - self.update_function_if_needed(function_name, &existing, config) + self.update_function_if_needed(function_name, &existing, deploy_config) .await } Err(e) => Err(e), @@ -112,7 +116,11 @@ impl LambdaDeployer { /// Deploy is a no-op when auto-deploy feature is not enabled. #[cfg(not(feature = "auto-deploy"))] - pub async fn deploy(&self, _config: &LambdaConfig) -> LambdaResult { + pub async fn deploy( + &self, + _function_name: &str, + _deploy_config: &LambdaDeployConfig, + ) -> LambdaResult { Err(LambdaError::Configuration( "auto-deploy feature is not enabled at compile time".into(), )) @@ -128,7 +136,7 @@ impl LambdaDeployer { &self, name: &str, role: &str, - config: &LambdaConfig, + config: &LambdaDeployConfig, ) -> LambdaResult { info!( function_name = %name, @@ -196,7 +204,7 @@ impl LambdaDeployer { &self, name: &str, existing: &GetFunctionOutput, - config: &LambdaConfig, + config: &LambdaDeployConfig, ) -> LambdaResult { let function_arn = existing .configuration() diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 0f1a698ad42..2bb347942b2 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -749,6 +749,7 @@ pub(crate) async fn search_partial_hits_phase( get_count_from_metadata(split_metadatas) } else if search_request.execution_mode == ExecutionMode::RemoteFunction as i32 && lambda_invoker.is_some() + && searcher_context.searcher_config.lambda.is_some() { // Execute via Lambda execute_leaf_search_via_lambda( @@ -756,7 +757,7 @@ pub(crate) async fn search_partial_hits_phase( indexes_metas_for_leaf_search, split_metadatas, lambda_invoker.as_ref().unwrap().as_ref(), - &searcher_context.searcher_config.lambda, + searcher_context.searcher_config.lambda.as_ref().unwrap(), ) .await? } else { diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index b107e79ba07..406306947a6 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1021,22 +1021,22 @@ async fn setup_searcher( // Initialize Lambda invoker if enabled let lambda_invoker: Option> = - if node_config.searcher_config.lambda.enabled { + if let Some(lambda_config) = &node_config.searcher_config.lambda { info!("initializing AWS Lambda invoker for leaf search"); // Auto-deploy Lambda function if configured - if node_config.searcher_config.lambda.auto_deploy { + if let Some(deploy_config) = &lambda_config.auto_deploy { info!("auto-deploying Lambda function"); let deployer = LambdaDeployer::new() .await .context("failed to create Lambda deployer")?; deployer - .deploy(&node_config.searcher_config.lambda) + .deploy(&lambda_config.function_name, deploy_config) .await .context("failed to deploy Lambda function")?; } - let invoker = AwsLambdaInvoker::new(&node_config.searcher_config.lambda) + let invoker = AwsLambdaInvoker::new(lambda_config) .await .context("failed to initialize AWS Lambda invoker")?; From aa3c16a4bcc130b7e8596ed9cc64ba0fc099561c Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 16:44:22 +0100 Subject: [PATCH 11/22] better config --- .../quickwit-config/src/node_config/mod.rs | 30 ++++++++----------- quickwit/quickwit-lambda/README.md | 6 ++-- quickwit/quickwit-lambda/build.rs | 4 +-- quickwit/quickwit-lambda/src/config.rs | 18 +++++------ quickwit/quickwit-lambda/src/context.rs | 2 +- quickwit/quickwit-lambda/src/deployer.rs | 6 ++-- quickwit/quickwit-lambda/src/error.rs | 4 ++- 7 files changed, 34 insertions(+), 36 deletions(-) diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index adef2e11651..1cf1b8ede36 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -302,13 +302,17 @@ pub struct SearcherConfig { pub warmup_single_split_initial_allocation: ByteSize, /// Lambda configuration for serverless leaf search execution. /// If set, enables Lambda execution for leaf search. + /// + /// If set, and Quickwit cannot access the Lambda (after a deploy attempt if + /// auto deploy is set up), Quickwit will log an error and + /// fail on startup. #[serde(default)] pub lambda: Option, } /// Configuration for AWS Lambda leaf search execution. #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -#[serde(deny_unknown_fields, default)] +#[serde(deny_unknown_fields)] pub struct LambdaConfig { /// AWS Lambda function name or ARN. #[serde(default = "LambdaConfig::default_function_name")] @@ -321,37 +325,29 @@ pub struct LambdaConfig { pub max_splits_per_invocation: usize, /// Auto-deploy configuration. If set, Quickwit will automatically deploy /// the Lambda function at startup. + /// If deploying a lambda fails, Quickwit will log an error and fail. #[serde(default)] pub auto_deploy: Option, } /// Configuration for automatic Lambda function deployment. #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] -#[serde(deny_unknown_fields, default)] +#[serde(deny_unknown_fields)] pub struct LambdaDeployConfig { /// IAM execution role ARN for the Lambda function. pub execution_role_arn: String, - /// Memory size for the Lambda function in MB. - #[serde(default = "LambdaDeployConfig::default_memory_size_mb")] - pub memory_size_mb: u32, + /// Memory size for the Lambda function. It will be rounded up to the nearest multiple of 1MiB. + #[serde(default = "LambdaDeployConfig::default_memory_size")] + pub memory_size: ByteSize, /// Timeout for Lambda invocations in seconds. #[serde(default = "LambdaDeployConfig::default_invocation_timeout_secs")] pub invocation_timeout_secs: u64, } -impl Default for LambdaDeployConfig { - fn default() -> Self { - Self { - execution_role_arn: String::new(), - memory_size_mb: Self::default_memory_size_mb(), - invocation_timeout_secs: Self::default_invocation_timeout_secs(), - } - } -} - impl LambdaDeployConfig { - fn default_memory_size_mb() -> u32 { - 1024 + fn default_memory_size() -> ByteSize { + // Empirically this implies between 4 and 6 vCPUs. + ByteSize::gib(5) } fn default_invocation_timeout_secs() -> u64 { 30 diff --git a/quickwit/quickwit-lambda/README.md b/quickwit/quickwit-lambda/README.md index 842a0c3ef7d..830c9cd10e1 100644 --- a/quickwit/quickwit-lambda/README.md +++ b/quickwit/quickwit-lambda/README.md @@ -34,7 +34,7 @@ searcher: # Auto-deploy settings (requires auto-deploy feature) auto_deploy: execution_role_arn: "arn:aws:iam::123456789:role/quickwit-lambda-role" - memory_size_mb: 1024 + memory_size: 5GiB invocation_timeout_secs: 30 ``` @@ -54,7 +54,7 @@ searcher: | Option | Type | Default | Description | |--------|------|---------|-------------| | `execution_role_arn` | string | required | IAM role ARN for the Lambda function | -| `memory_size_mb` | int | `1024` | Lambda memory allocation in MB | +| `memory_size` | ByteSize | `5GiB` | Lambda memory allocation | | `invocation_timeout_secs` | int | `30` | Timeout for Lambda invocations in seconds | ## Auto-Deploy Feature @@ -221,7 +221,7 @@ If Quickwit fails to start with Lambda enabled: ### Runtime Issues 1. **Timeouts**: Increase `invocation_timeout_secs` -2. **Memory errors**: Increase `memory_size_mb` +2. **Memory errors**: Increase `memory_size` 3. **Throttling**: AWS Lambda has concurrency limits - check your account limits ### Build Issues diff --git a/quickwit/quickwit-lambda/build.rs b/quickwit/quickwit-lambda/build.rs index 12406e52ce5..3dfe3493254 100644 --- a/quickwit/quickwit-lambda/build.rs +++ b/quickwit/quickwit-lambda/build.rs @@ -92,8 +92,8 @@ fn create_lambda_zip(binary_path: &std::path::Path, zip_path: &std::path::Path) use std::fs::File; use std::io::Read; - use zip::write::FileOptions; use zip::ZipWriter; + use zip::write::FileOptions; let mut binary_data = Vec::new(); File::open(binary_path) @@ -127,8 +127,8 @@ fn create_lambda_zip(binary_path: &std::path::Path, zip_path: &std::path::Path) fn create_placeholder_zip(zip_path: &std::path::Path) { use std::fs::File; - use zip::write::FileOptions; use zip::ZipWriter; + use zip::write::FileOptions; let zip_file = File::create(zip_path).expect("Failed to create placeholder zip file"); let mut zip = ZipWriter::new(zip_file); diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda/src/config.rs index dc69dea727a..3f8eda1cca7 100644 --- a/quickwit/quickwit-lambda/src/config.rs +++ b/quickwit/quickwit-lambda/src/config.rs @@ -29,10 +29,10 @@ pub struct LambdaSearcherConfig { } impl LambdaSearcherConfig { - pub fn try_from_env() -> anyhow::Result { - let memory_mb: usize = quickwit_common::get_from_env_opt("AWS_LAMBDA_FUNCTION_MEMORY_SIZE", false) - .context("could not get aws lambda function memory size from ENV")?; + let memory_mb: usize = + quickwit_common::get_from_env_opt("AWS_LAMBDA_FUNCTION_MEMORY_SIZE", false) + .context("could not get aws lambda function memory size from ENV")?; Ok(LambdaSearcherConfig::for_memory(memory_mb)) } /// Create a Lambda-optimized searcher config based on the allocated memory. @@ -47,9 +47,9 @@ impl LambdaSearcherConfig { } } -impl Default for LambdaSearcherConfig { - fn default() -> Self { - // Default to 1024 MB Lambda - Self::for_memory(1024) - } -} +// impl Default for LambdaSearcherConfig { +// fn default() -> Self { +// // Default to 1024 MB Lambda +// Self::for_memory(1024) +// } +// } diff --git a/quickwit/quickwit-lambda/src/context.rs b/quickwit/quickwit-lambda/src/context.rs index cd660d438d7..5381994aedb 100644 --- a/quickwit/quickwit-lambda/src/context.rs +++ b/quickwit/quickwit-lambda/src/context.rs @@ -30,7 +30,7 @@ pub struct LambdaSearcherContext { impl LambdaSearcherContext { /// Create a new Lambda searcher context from environment variables. pub fn try_from_env() -> anyhow::Result { - info!("Initializing Lambda searcher context"); + info!("initializing Lambda searcher context"); let config = LambdaSearcherConfig::try_from_env()?; let searcher_config = create_searcher_config(&config); diff --git a/quickwit/quickwit-lambda/src/deployer.rs b/quickwit/quickwit-lambda/src/deployer.rs index fcec6a4c48f..9fd61e4294e 100644 --- a/quickwit/quickwit-lambda/src/deployer.rs +++ b/quickwit/quickwit-lambda/src/deployer.rs @@ -141,7 +141,7 @@ impl LambdaDeployer { info!( function_name = %name, role = %role, - memory_mb = config.memory_size_mb, + memory_size = %config.memory_size, timeout_secs = config.invocation_timeout_secs, "creating Lambda function" ); @@ -162,7 +162,7 @@ impl LambdaDeployer { .zip_file(Blob::new(LAMBDA_BINARY)) .build(), ) - .memory_size(config.memory_size_mb as i32) + .memory_size((config.memory_size.as_u64() / (1024 * 1024)) as i32) .timeout(config.invocation_timeout_secs as i32) .environment(environment) .description(format!( @@ -245,7 +245,7 @@ impl LambdaDeployer { self.client .update_function_configuration() .function_name(name) - .memory_size(config.memory_size_mb as i32) + .memory_size((config.memory_size.as_u64() / (1024 * 1024)) as i32) .timeout(config.invocation_timeout_secs as i32) .environment(self.build_environment()) .description(format!( diff --git a/quickwit/quickwit-lambda/src/error.rs b/quickwit/quickwit-lambda/src/error.rs index f40ff93a4b8..5c33f162942 100644 --- a/quickwit/quickwit-lambda/src/error.rs +++ b/quickwit/quickwit-lambda/src/error.rs @@ -51,7 +51,9 @@ impl fmt::Display for LambdaError { LambdaError::FunctionError(msg) => write!(f, "Lambda function error: {}", msg), LambdaError::Configuration(msg) => write!(f, "Configuration error: {}", msg), LambdaError::Internal(msg) => write!(f, "Internal error: {}", msg), - LambdaError::ResourceConflict => write!(f, "Resource conflict: function already exists"), + LambdaError::ResourceConflict => { + write!(f, "Resource conflict: function already exists") + } LambdaError::Deployment(msg) => write!(f, "Deployment error: {}", msg), LambdaError::NotFound(name) => write!(f, "Lambda function not found: {}", name), } From bcae2c7758ad1ebb59af0c0ac1ec689c4f6bf962 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 17:18:15 +0100 Subject: [PATCH 12/22] blop --- quickwit/quickwit-cli/Cargo.toml | 3 ++ .../quickwit-config/src/node_config/mod.rs | 4 -- quickwit/quickwit-lambda/src/deployer.rs | 44 ++++++------------- quickwit/quickwit-lambda/src/invoker.rs | 4 -- quickwit/quickwit-lambda/src/lib.rs | 21 +++++++++ quickwit/quickwit-serve/Cargo.toml | 3 ++ quickwit/quickwit-serve/src/lib.rs | 9 +--- 7 files changed, 42 insertions(+), 46 deletions(-) diff --git a/quickwit/quickwit-cli/Cargo.toml b/quickwit/quickwit-cli/Cargo.toml index c595cb7e90a..6e0e09868e7 100644 --- a/quickwit/quickwit-cli/Cargo.toml +++ b/quickwit/quickwit-cli/Cargo.toml @@ -102,6 +102,7 @@ release-feature-set = [ "quickwit-indexing/pulsar", "quickwit-indexing/sqs", "quickwit-indexing/vrl", + "quickwit-serve/lambda-auto-deploy", "quickwit-storage/azure", "quickwit-storage/gcs", "quickwit-metastore/postgres", @@ -116,6 +117,7 @@ release-feature-vendored-set = [ "quickwit-indexing/sqs", "quickwit-indexing/vrl", "quickwit-indexing/vendored-kafka", + "quickwit-serve/lambda-auto-deploy", "quickwit-storage/azure", "quickwit-storage/gcs", "quickwit-metastore/postgres", @@ -129,6 +131,7 @@ release-macos-feature-vendored-set = [ "quickwit-indexing/sqs", "quickwit-indexing/vrl", "quickwit-indexing/vendored-kafka-macos", + "quickwit-serve/lambda-auto-deploy", "quickwit-storage/azure", "quickwit-storage/gcs", "quickwit-metastore/postgres", diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index 1cf1b8ede36..82dbd2e4f6f 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -317,9 +317,6 @@ pub struct LambdaConfig { /// AWS Lambda function name or ARN. #[serde(default = "LambdaConfig::default_function_name")] pub function_name: String, - /// Optional function qualifier (alias or version). - #[serde(default)] - pub function_qualifier: Option, /// Maximum number of splits per Lambda invocation. #[serde(default = "LambdaConfig::default_max_splits_per_invocation")] pub max_splits_per_invocation: usize, @@ -358,7 +355,6 @@ impl Default for LambdaConfig { fn default() -> Self { Self { function_name: Self::default_function_name(), - function_qualifier: None, max_splits_per_invocation: Self::default_max_splits_per_invocation(), auto_deploy: None, } diff --git a/quickwit/quickwit-lambda/src/deployer.rs b/quickwit/quickwit-lambda/src/deployer.rs index 9fd61e4294e..4e6a3cd3599 100644 --- a/quickwit/quickwit-lambda/src/deployer.rs +++ b/quickwit/quickwit-lambda/src/deployer.rs @@ -17,21 +17,15 @@ //! This module provides functionality to automatically deploy or update //! the Lambda function used for leaf search operations. -#[cfg(feature = "auto-deploy")] use std::collections::HashMap; use aws_sdk_lambda::Client as LambdaClient; -#[cfg(feature = "auto-deploy")] use aws_sdk_lambda::error::SdkError; -#[cfg(feature = "auto-deploy")] use aws_sdk_lambda::operation::create_function::CreateFunctionError; -#[cfg(feature = "auto-deploy")] use aws_sdk_lambda::operation::get_function::GetFunctionOutput; -#[cfg(feature = "auto-deploy")] use aws_sdk_lambda::primitives::Blob; -#[cfg(feature = "auto-deploy")] use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; -#[cfg(feature = "auto-deploy")] +use quickwit_config::LambdaDeployConfig; use tracing::{debug, info, warn}; use crate::config::LambdaDeployConfig; @@ -39,15 +33,12 @@ use crate::error::{LambdaError, LambdaResult}; /// Embedded Lambda binary (arm64, compressed). /// This is included at compile time when the `auto-deploy` feature is enabled. -#[cfg(feature = "auto-deploy")] const LAMBDA_BINARY: &[u8] = include_bytes!(concat!(env!("OUT_DIR"), "/lambda_bootstrap.zip")); /// Version tag key used to track deployed Quickwit version. -#[cfg(feature = "auto-deploy")] const VERSION_TAG_KEY: &str = "quickwit_version"; /// Description prefix for auto-deployed Lambda functions. -#[cfg(feature = "auto-deploy")] const FUNCTION_DESCRIPTION: &str = "Quickwit Lambda leaf search handler"; /// Lambda function deployer. @@ -55,7 +46,6 @@ const FUNCTION_DESCRIPTION: &str = "Quickwit Lambda leaf search handler"; /// Handles creating and updating Lambda functions for the auto-deploy feature. /// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. pub struct LambdaDeployer { - #[allow(dead_code)] client: LambdaClient, } @@ -68,7 +58,6 @@ impl LambdaDeployer { } /// Create a new Lambda deployer with a custom client. - #[allow(dead_code)] pub fn with_client(client: LambdaClient) -> Self { Self { client } } @@ -77,7 +66,6 @@ impl LambdaDeployer { /// /// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. /// Returns the function ARN. - #[cfg(feature = "auto-deploy")] pub async fn deploy( &self, function_name: &str, @@ -114,24 +102,13 @@ impl LambdaDeployer { } } - /// Deploy is a no-op when auto-deploy feature is not enabled. - #[cfg(not(feature = "auto-deploy"))] - pub async fn deploy( - &self, - _function_name: &str, - _deploy_config: &LambdaDeployConfig, - ) -> LambdaResult { - Err(LambdaError::Configuration( - "auto-deploy feature is not enabled at compile time".into(), - )) - } + /// Create the Lambda function. /// /// Note: CreateFunction is idempotent - if the function already exists, AWS returns /// ResourceConflictException. Multiple Quickwit nodes starting simultaneously is safe; /// one will succeed and others will fall back to update_function. - #[cfg(feature = "auto-deploy")] async fn create_function( &self, name: &str, @@ -199,7 +176,6 @@ impl LambdaDeployer { /// /// Compares the deployed version tag with the current Quickwit version /// and updates if they differ. - #[cfg(feature = "auto-deploy")] async fn update_function_if_needed( &self, name: &str, @@ -278,7 +254,6 @@ impl LambdaDeployer { } /// Wait for function update to complete. - #[cfg(feature = "auto-deploy")] async fn wait_for_update_complete(&self, name: &str) -> LambdaResult<()> { // Poll until the function state is Active and LastUpdateStatus is Successful for _ in 0..60 { @@ -319,7 +294,6 @@ impl LambdaDeployer { } /// Check if the function needs to be updated based on version tag. - #[cfg(feature = "auto-deploy")] fn needs_update(&self, existing: &GetFunctionOutput) -> bool { let current_version = env!("CARGO_PKG_VERSION"); @@ -370,7 +344,6 @@ impl LambdaDeployer { } /// Build environment variables for the Lambda function. - #[cfg(feature = "auto-deploy")] fn build_environment(&self) -> Environment { let mut env_vars = HashMap::new(); // Set reasonable defaults for logging @@ -381,7 +354,6 @@ impl LambdaDeployer { } /// Build tags for the Lambda function. - #[cfg(feature = "auto-deploy")] fn build_tags(&self) -> HashMap { let mut tags = HashMap::new(); tags.insert( @@ -393,7 +365,17 @@ impl LambdaDeployer { } } -#[cfg(all(test, feature = "auto-deploy"))] +pub async fn deploy( + function_name: &str, + deploy_config: &LambdaDeployConfig, +) -> LambdaResult { + let lambda_deployer = LambdaDeployer::new().await?; + let lambda_arn = lambda_deployer.deploy(function_name, deploy_config).await?; + info!("successfully deployed lambda function `{}`", lambda_arn); + Ok(lambda_arn) +} + +#[cfg(test)] mod tests { use super::*; diff --git a/quickwit/quickwit-lambda/src/invoker.rs b/quickwit/quickwit-lambda/src/invoker.rs index 3e5e2499595..9af909b01b1 100644 --- a/quickwit/quickwit-lambda/src/invoker.rs +++ b/quickwit/quickwit-lambda/src/invoker.rs @@ -30,7 +30,6 @@ use crate::handler::{LeafSearchPayload, LeafSearchResponsePayload}; pub struct AwsLambdaInvoker { client: LambdaClient, function_name: String, - qualifier: Option, } impl AwsLambdaInvoker { @@ -42,7 +41,6 @@ impl AwsLambdaInvoker { Ok(Self { client, function_name: config.function_name.clone(), - qualifier: config.function_qualifier.clone(), }) } @@ -50,12 +48,10 @@ impl AwsLambdaInvoker { pub fn with_client( client: LambdaClient, function_name: String, - qualifier: Option, ) -> Self { Self { client, function_name, - qualifier, } } diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda/src/lib.rs index 923b526721a..f5c7c8c77b4 100644 --- a/quickwit/quickwit-lambda/src/lib.rs +++ b/quickwit/quickwit-lambda/src/lib.rs @@ -28,7 +28,28 @@ mod invoker; pub use config::{LambdaConfig, LambdaSearcherConfig}; pub use context::LambdaSearcherContext; + +#[cfg(feature = "auto-deploy")] +mod deployer; +#[cfg(feature = "auto-deploy")] pub use deployer::LambdaDeployer; + +#[cfg(feature = "auto-deploy")] +mod deployer; +#[cfg(feature = "auto-deploy")] +pub use deployer::deploy; + pub use error::{LambdaError, LambdaResult}; pub use handler::{LeafSearchPayload, LeafSearchResponsePayload, handle_leaf_search}; pub use invoker::AwsLambdaInvoker; + +/// Deploy is a no-op when auto-deploy feature is not enabled. +#[cfg(not(feature = "auto-deploy"))] +pub async fn deploy( + _function_name: &str, + _deploy_config: &quickwit_config::LambdaDeployConfig, +) -> LambdaResult { + Err(LambdaError::Configuration( + "auto-deploy feature is not enabled at compile time".into(), + )) +} diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index eadc2fb8c80..07ac742ac19 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -110,6 +110,9 @@ pprof = [ jemalloc-profiled = [ "quickwit-common/jemalloc-profiled" ] +lambda-auto-deploy = [ + "quickwit-lambda/auto-deploy" +] testsuite = [] sqs-for-tests = [ "quickwit-indexing/sqs", diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 406306947a6..f455d3b8974 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1027,13 +1027,8 @@ async fn setup_searcher( // Auto-deploy Lambda function if configured if let Some(deploy_config) = &lambda_config.auto_deploy { info!("auto-deploying Lambda function"); - let deployer = LambdaDeployer::new() - .await - .context("failed to create Lambda deployer")?; - deployer - .deploy(&lambda_config.function_name, deploy_config) - .await - .context("failed to deploy Lambda function")?; + quickwit_lambda::deploy(&lambda_config.function_name, deploy_config) + .context("failed to deploy lambda function")?; } let invoker = AwsLambdaInvoker::new(lambda_config) From 1e18cd526fffcce05ecc05d664314f4cecbc650b Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 28 Jan 2026 17:47:43 +0100 Subject: [PATCH 13/22] blop --- .github/workflows/publish_lambda.yaml | 70 ++++++++ LICENSE-3rdparty.csv | 6 + quickwit/Cargo.lock | 56 ++++-- quickwit/Cargo.toml | 1 + quickwit/quickwit-common/src/rate_limiter.rs | 1 - quickwit/quickwit-config/Cargo.toml | 1 + .../resources/tests/node_config/quickwit.json | 9 + .../resources/tests/node_config/quickwit.toml | 9 + .../resources/tests/node_config/quickwit.yaml | 7 + .../quickwit-config/src/node_config/mod.rs | 2 +- .../src/node_config/serialize.rs | 12 +- quickwit/quickwit-lambda/Cargo.toml | 18 +- quickwit/quickwit-lambda/build.rs | 159 ++++++------------ .../quickwit-lambda/src/bin/leaf_search.rs | 6 + quickwit/quickwit-lambda/src/config.rs | 29 ++-- quickwit/quickwit-lambda/src/deployer.rs | 4 - quickwit/quickwit-lambda/src/invoker.rs | 41 ++--- quickwit/quickwit-lambda/src/lib.rs | 10 +- quickwit/quickwit-serve/src/lib.rs | 45 +++-- 19 files changed, 279 insertions(+), 207 deletions(-) create mode 100644 .github/workflows/publish_lambda.yaml diff --git a/.github/workflows/publish_lambda.yaml b/.github/workflows/publish_lambda.yaml new file mode 100644 index 00000000000..3700188d734 --- /dev/null +++ b/.github/workflows/publish_lambda.yaml @@ -0,0 +1,70 @@ +name: Build Lambda binary + +on: + push: + branches: + - lambda # This is temporary + workflow_dispatch: + inputs: + version: + description: 'Version tag (e.g., v0.8.0)' + required: false + default: 'dev' + +permissions: + contents: read + +jobs: + build-lambda: + name: Build Lambda ARM64 + runs-on: ubuntu-latest + permissions: + contents: write + actions: write + steps: + - uses: actions/checkout@8e8c483db84b4bee98b60c0593521ed34d9990e8 # v6.0.1 + + - name: Set version + run: | + if [ "${{ github.event.inputs.version }}" = "dev" ]; then + echo "ASSET_VERSION=dev-$(git rev-parse --short HEAD)" >> $GITHUB_ENV + else + echo "ASSET_VERSION=${{ github.event.inputs.version }}" >> $GITHUB_ENV + fi + + - name: Install rustup + run: curl https://sh.rustup.rs -sSf | sh -s -- --default-toolchain none -y + + - name: Install cross + run: cargo install cross + + - name: Retrieve and export commit date, hash, and tags + run: | + echo "QW_COMMIT_DATE=$(TZ=UTC0 git log -1 --format=%cd --date=format-local:%Y-%m-%dT%H:%M:%SZ)" >> $GITHUB_ENV + echo "QW_COMMIT_HASH=$(git rev-parse HEAD)" >> $GITHUB_ENV + echo "QW_COMMIT_TAGS=$(git tag --points-at HEAD | tr '\n' ',')" >> $GITHUB_ENV + + - name: Build Lambda binary + run: cross build --release --features lambda-release --target aarch64-unknown-linux-gnu -p quickwit-lambda --bin quickwit-aws-lambda-leaf-search + env: + QW_COMMIT_DATE: ${{ env.QW_COMMIT_DATE }} + QW_COMMIT_HASH: ${{ env.QW_COMMIT_HASH }} + QW_COMMIT_TAGS: ${{ env.QW_COMMIT_TAGS }} + working-directory: ./quickwit + + - name: Create Lambda zip + run: | + cd quickwit/target/aarch64-unknown-linux-gnu/release + cp quickwit-aws-lambda-leaf-search bootstrap + zip quickwit-aws-lambda-${{ env.ASSET_VERSION }}-aarch64.zip bootstrap + mv quickwit-aws-lambda-${{ env.ASSET_VERSION }}-aarch64.zip ../../../../ + + - name: Upload to GitHub release + uses: quickwit-inc/upload-to-github-release@v1 + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + with: + file: quickwit-aws-lambda-${{ env.ASSET_VERSION }}-aarch64.zip + overwrite: true + draft: true + tag_name: ${{ env.ASSET_VERSION }} diff --git a/LICENSE-3rdparty.csv b/LICENSE-3rdparty.csv index 8f199a01910..1a640e966b1 100644 --- a/LICENSE-3rdparty.csv +++ b/LICENSE-3rdparty.csv @@ -21,6 +21,8 @@ arrayvec,https://github.com/bluss/arrayvec,MIT OR Apache-2.0,bluss assert-json-diff,https://github.com/davidpdrsn/assert-json-diff,MIT,David Pedersen async-compression,https://github.com/Nullus157/async-compression,MIT OR Apache-2.0,"Wim Looman , Allen Bui " async-speed-limit,https://github.com/tikv/async-speed-limit,MIT OR Apache-2.0,The TiKV Project Developers +async-stream,https://github.com/tokio-rs/async-stream,MIT,Carl Lerche +async-stream-impl,https://github.com/tokio-rs/async-stream,MIT,Carl Lerche async-trait,https://github.com/dtolnay/async-trait,MIT OR Apache-2.0,David Tolnay atomic-waker,https://github.com/smol-rs/atomic-waker,Apache-2.0 OR MIT,"Stjepan Glavina , Contributors to futures-rs" aws-config,https://github.com/smithy-lang/smithy-rs,Apache-2.0,"AWS Rust SDK Team , Russell Cohen " @@ -28,6 +30,7 @@ aws-credential-types,https://github.com/smithy-lang/smithy-rs,Apache-2.0,AWS Rus aws-lc-rs,https://github.com/aws/aws-lc-rs,ISC AND (Apache-2.0 OR ISC),AWS-LibCrypto aws-lc-sys,https://github.com/aws/aws-lc-rs,ISC AND (Apache-2.0 OR ISC) AND OpenSSL,AWS-LC aws-runtime,https://github.com/smithy-lang/smithy-rs,Apache-2.0,AWS Rust SDK Team +aws-sdk-lambda,https://github.com/awslabs/aws-sdk-rust,Apache-2.0,"AWS Rust SDK Team , Russell Cohen " aws-sdk-s3,https://github.com/awslabs/aws-sdk-rust,Apache-2.0,"AWS Rust SDK Team , Russell Cohen " aws-sdk-sso,https://github.com/awslabs/aws-sdk-rust,Apache-2.0,"AWS Rust SDK Team , Russell Cohen " aws-sdk-ssooidc,https://github.com/awslabs/aws-sdk-rust,Apache-2.0,"AWS Rust SDK Team , Russell Cohen " @@ -235,6 +238,8 @@ jiff-static,https://github.com/BurntSushi/jiff,Unlicense OR MIT,Andrew Gallant < jobserver,https://github.com/rust-lang/jobserver-rs,MIT OR Apache-2.0,Alex Crichton js-sys,https://github.com/wasm-bindgen/wasm-bindgen/tree/master/crates/js-sys,MIT OR Apache-2.0,The wasm-bindgen Developers json_comments,https://github.com/tmccombs/json-comments-rs,Apache-2.0,Thayne McCombs +lambda_runtime,https://github.com/awslabs/aws-lambda-rust-runtime,Apache-2.0,"David Calavera , Harold Sun " +lambda_runtime_api_client,https://github.com/awslabs/aws-lambda-rust-runtime,Apache-2.0,"David Calavera , Harold Sun " lazy_static,https://github.com/rust-lang-nursery/lazy-static.rs,MIT OR Apache-2.0,Marvin Löbel levenshtein_automata,https://github.com/tantivy-search/levenshtein-automata,MIT,Paul Masurel libc,https://github.com/rust-lang/libc,MIT OR Apache-2.0,The Rust Project Developers @@ -424,6 +429,7 @@ serde_core,https://github.com/serde-rs/serde,MIT OR Apache-2.0,"Erick Tryzelaar serde_derive,https://github.com/serde-rs/serde,MIT OR Apache-2.0,"Erick Tryzelaar , David Tolnay " serde_json,https://github.com/serde-rs/json,MIT OR Apache-2.0,"Erick Tryzelaar , David Tolnay " serde_json_borrow,https://github.com/PSeitz/serde_json_borrow,MIT,Pascal Seitz +serde_path_to_error,https://github.com/dtolnay/path-to-error,MIT OR Apache-2.0,David Tolnay serde_qs,https://github.com/samscott89/serde_qs,MIT OR Apache-2.0,Sam Scott serde_spanned,https://github.com/toml-rs/toml,MIT OR Apache-2.0,The serde_spanned Authors serde_urlencoded,https://github.com/nox/serde_urlencoded,MIT OR Apache-2.0,Anthony Ramine diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 7e6c433900a..4ce715ba9c9 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -4552,7 +4552,7 @@ dependencies = [ "lindera-core", "once_cell", "tar", - "ureq", + "ureq 2.12.1", ] [[package]] @@ -4637,7 +4637,7 @@ dependencies = [ "lindera-ipadic-builder", "once_cell", "tar", - "ureq", + "ureq 2.12.1", ] [[package]] @@ -4696,7 +4696,7 @@ dependencies = [ "lindera-ko-dic-builder", "once_cell", "tar", - "ureq", + "ureq 2.12.1", ] [[package]] @@ -7360,6 +7360,7 @@ dependencies = [ "bytesize", "lambda_runtime", "mockall", + "openssl", "prost 0.14.1", "quickwit-aws", "quickwit-common", @@ -7375,7 +7376,7 @@ dependencies = [ "tokio", "tracing", "tracing-subscriber", - "zip", + "ureq 3.1.4", ] [[package]] @@ -10690,6 +10691,35 @@ dependencies = [ "webpki-roots 0.26.11", ] +[[package]] +name = "ureq" +version = "3.1.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d39cb1dbab692d82a977c0392ffac19e188bd9186a9f32806f0aaa859d75585a" +dependencies = [ + "base64 0.22.1", + "flate2", + "log", + "percent-encoding", + "rustls 0.23.36", + "rustls-pki-types", + "ureq-proto", + "utf-8", + "webpki-roots 1.0.5", +] + +[[package]] +name = "ureq-proto" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d81f9efa9df032be5934a46a068815a10a042b494b6a58cb0a1a97bb5467ed6f" +dependencies = [ + "base64 0.22.1", + "http 1.4.0", + "httparse", + "log", +] + [[package]] name = "url" version = "2.5.8" @@ -10719,6 +10749,12 @@ dependencies = [ "winapi 0.2.8", ] +[[package]] +name = "utf-8" +version = "0.7.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09cc8ee72d2a9becf2f2febe0205bbed8fc6615b7cb429ad062dc7b7ddd036a9" + [[package]] name = "utf8-ranges" version = "1.0.5" @@ -11804,18 +11840,6 @@ dependencies = [ "syn 2.0.114", ] -[[package]] -name = "zip" -version = "0.6.6" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "760394e246e4c28189f19d488c058bf16f564016aefac5d32bb1f3b51d5e9261" -dependencies = [ - "byteorder", - "crc32fast", - "crossbeam-utils", - "flate2", -] - [[package]] name = "zlib-rs" version = "0.5.5" diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index 7bf6ad55fce..ae3670a2bcc 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -296,6 +296,7 @@ tracing-subscriber = { version = "0.3", features = [ ttl_cache = "0.5" typetag = "0.2" ulid = "1.2" +ureq = "3" username = "0.2" # We cannot upgrade to utoipa 5.0+ due to significant breaking changes: # 1. The `OpenApi` struct structure changed (fields are private), breaking our manual merging logic in openapi.rs diff --git a/quickwit/quickwit-common/src/rate_limiter.rs b/quickwit/quickwit-common/src/rate_limiter.rs index 739ce8428c0..5b7c2e4e92a 100644 --- a/quickwit/quickwit-common/src/rate_limiter.rs +++ b/quickwit/quickwit-common/src/rate_limiter.rs @@ -33,7 +33,6 @@ pub struct RateLimiterSettings { pub refill_period: Duration, } -#[cfg(any(test, feature = "testsuite"))] impl Default for RateLimiterSettings { fn default() -> Self { // 10 MB burst limit. diff --git a/quickwit/quickwit-config/Cargo.toml b/quickwit/quickwit-config/Cargo.toml index 7cf75818444..a1877661490 100644 --- a/quickwit/quickwit-config/Cargo.toml +++ b/quickwit/quickwit-config/Cargo.toml @@ -43,6 +43,7 @@ quickwit-proto = { workspace = true } tokio = { workspace = true } quickwit-proto = { workspace = true, features = ["testsuite"] } +quickwit-common = { workspace = true, features = ["testsuite"] } [features] testsuite = [] diff --git a/quickwit/quickwit-config/resources/tests/node_config/quickwit.json b/quickwit/quickwit-config/resources/tests/node_config/quickwit.json index 01def63b10e..b0d04650d32 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.json +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.json @@ -70,6 +70,15 @@ "min_throughtput_bytes_per_secs": 100000, "timeout_millis": 2000, "max_num_retries": 2 + }, + "lambda": { + "function_name": "quickwit-lambda-leaf-search", + "max_splits_per_invocation": 10, + "auto_deploy": { + "execution_role_arn": "arn:aws:iam::123456789012:role/quickwit-lambda-role", + "memory_size": "5 GiB", + "invocation_timeout_secs": 15 + } } }, "jaeger": { diff --git a/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml b/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml index 0b4e0c30229..16c4b056991 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml @@ -62,6 +62,15 @@ min_throughtput_bytes_per_secs = 100000 timeout_millis = 2000 max_num_retries = 2 +[searcher.lambda] +function_name = "quickwit-lambda-leaf-search" +max_splits_per_invocation = 10 + +[searcher.lambda.auto_deploy] +execution_role_arn = "arn:aws:iam::123456789012:role/quickwit-lambda-role" +memory_size = "5 GiB" +invocation_timeout_secs = 15 + [jaeger] enable_endpoint = true lookback_period_hours = 24 diff --git a/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml b/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml index cb16052fbd1..f000bd76c72 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml @@ -64,6 +64,13 @@ searcher: min_throughtput_bytes_per_secs: 100000 timeout_millis: 2000 max_num_retries: 2 + lambda: + function_name: quickwit-lambda-leaf-search + max_splits_per_invocation: 10 + auto_deploy: + execution_role_arn: arn:aws:iam::123456789012:role/quickwit-lambda-role + memory_size: 5 GiB + invocation_timeout_secs: 15 jaeger: enable_endpoint: true diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index 82dbd2e4f6f..13232158cad 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -347,7 +347,7 @@ impl LambdaDeployConfig { ByteSize::gib(5) } fn default_invocation_timeout_secs() -> u64 { - 30 + 15 } } diff --git a/quickwit/quickwit-config/src/node_config/serialize.rs b/quickwit/quickwit-config/src/node_config/serialize.rs index 7d3b5f2c64b..f072e7a9760 100644 --- a/quickwit/quickwit-config/src/node_config/serialize.rs +++ b/quickwit/quickwit-config/src/node_config/serialize.rs @@ -542,8 +542,8 @@ mod tests { use itertools::Itertools; use super::*; - use crate::CacheConfig; use crate::storage_config::StorageBackendFlavor; + use crate::{CacheConfig, LambdaConfig, LambdaDeployConfig}; fn get_config_filepath(config_filename: &str) -> String { format!( @@ -687,6 +687,16 @@ mod tests { }), warmup_memory_budget: ByteSize::gb(100), warmup_single_split_initial_allocation: ByteSize::gb(1), + lambda: Some(LambdaConfig { + function_name: "quickwit-lambda-leaf-search".to_string(), + max_splits_per_invocation: 10, + auto_deploy: Some(LambdaDeployConfig { + execution_role_arn: "arn:aws:iam::123456789012:role/quickwit-lambda-role" + .to_string(), + memory_size: ByteSize::gib(5), + invocation_timeout_secs: 15, + }), + }), } ); assert_eq!( diff --git a/quickwit/quickwit-lambda/Cargo.toml b/quickwit/quickwit-lambda/Cargo.toml index bd1bb6359f8..988202d8502 100644 --- a/quickwit/quickwit-lambda/Cargo.toml +++ b/quickwit/quickwit-lambda/Cargo.toml @@ -28,6 +28,8 @@ tracing-subscriber = { workspace = true, features = ["env-filter", "json"] } aws-config = { workspace = true } aws-sdk-lambda = { workspace = true } +openssl = { workspace = true, optional = true } + quickwit-aws = { workspace = true } quickwit-common = { workspace = true } quickwit-config = { workspace = true } @@ -41,16 +43,26 @@ mockall = { workspace = true } tempfile = { workspace = true } [[bin]] -name = "quickwit-lambda-leaf-search" +name = "quickwit-aws-lambda-leaf-search" path = "src/bin/leaf_search.rs" [build-dependencies] # Build dependencies for auto-deploy feature. -zip = { workspace = true, optional = true } +ureq = { workspace = true, optional = true } [features] default = [] testsuite = [] + # Enable automatic Lambda function deployment at startup. # When enabled, the Lambda binary is embedded at compile time. -auto-deploy = ["dep:zip"] +auto-deploy = ["dep:ureq"] + +# Keep this in sync with quickwit-cli! +lambda-release = [ + # The vendored OpenSSL will be compiled from source during the + # build, avoiding the pkg-config dependency issue during + # cross-compilation. + "openssl/vendored", + # "quickwit-doc-mapper/multilang", +] diff --git a/quickwit/quickwit-lambda/build.rs b/quickwit/quickwit-lambda/build.rs index 3dfe3493254..44bfc404eec 100644 --- a/quickwit/quickwit-lambda/build.rs +++ b/quickwit/quickwit-lambda/build.rs @@ -15,12 +15,10 @@ //! Build script for quickwit-lambda. //! //! When the `auto-deploy` feature is enabled, this script: -//! 1. Looks for a pre-built Lambda binary at a known location -//! 2. Creates a zip file containing the binary named "bootstrap" -//! 3. Places the zip in OUT_DIR for embedding via include_bytes! +//! 1. Downloads the pre-built Lambda zip from a GitHub release +//! 2. Places the zip in OUT_DIR for embedding via include_bytes! //! -//! The Lambda binary should be pre-built in CI for the aarch64-unknown-linux-musl -//! target and placed in the expected location. +//! The Lambda binary is built separately in CI and published as a GitHub release. fn main() { #[cfg(feature = "auto-deploy")] @@ -30,127 +28,68 @@ fn main() { println!("cargo:rerun-if-changed=build.rs"); } +/// URL to download the pre-built Lambda zip from GitHub releases. +/// This should be updated when a new Lambda binary is released. +#[cfg(feature = "auto-deploy")] +const LAMBDA_ZIP_URL: &str = + "https://github.com/quickwit-oss/quickwit/releases/download/lambda-506751fb/quickwit-aws-lambda--aarch64.zip"; + +/// AWS Lambda direct upload limit is 50MB. +/// Larger artifacts must be uploaded via S3. +#[cfg(feature = "auto-deploy")] +const MAX_LAMBDA_ZIP_SIZE: usize = 50 * 1024 * 1024; + #[cfg(feature = "auto-deploy")] fn auto_deploy_build() { use std::env; + use std::fs::File; + use std::io::Write; use std::path::PathBuf; println!("cargo:rerun-if-changed=build.rs"); - println!("cargo:rerun-if-env-changed=QUICKWIT_LAMBDA_BINARY_PATH"); + println!("cargo:rerun-if-env-changed=QUICKWIT_LAMBDA_ZIP_URL"); let out_dir = PathBuf::from(env::var("OUT_DIR").expect("OUT_DIR not set")); let zip_path = out_dir.join("lambda_bootstrap.zip"); - // Look for the pre-built Lambda binary in order of preference: - // 1. Environment variable QUICKWIT_LAMBDA_BINARY_PATH - // 2. Target directory (for local builds with cross-compilation) - // 3. Fallback to creating a placeholder for development - - let binary_path = if let Ok(path) = env::var("QUICKWIT_LAMBDA_BINARY_PATH") { - println!( - "cargo:warning=Using Lambda binary from QUICKWIT_LAMBDA_BINARY_PATH: {}", - path - ); - Some(PathBuf::from(path)) - } else { - // Try to find in target directory - let workspace_root = PathBuf::from(env::var("CARGO_MANIFEST_DIR").unwrap()) - .parent() - .unwrap() - .parent() - .unwrap() - .to_path_buf(); - - let potential_paths = [ - workspace_root - .join("target/aarch64-unknown-linux-musl/release/quickwit-lambda-leaf-search"), - workspace_root.join("target/lambda/quickwit-lambda-leaf-search/bootstrap"), - workspace_root.join("lambda_bootstrap"), - ]; + // Allow overriding the URL via environment variable + let url = env::var("QUICKWIT_LAMBDA_ZIP_URL").unwrap_or_else(|_| LAMBDA_ZIP_URL.to_string()); - potential_paths.into_iter().find(|p| p.exists()) - }; + println!("cargo:warning=Downloading Lambda zip from: {}", url); - match binary_path { - Some(path) => { - println!("cargo:warning=Packaging Lambda binary from: {:?}", path); - create_lambda_zip(&path, &zip_path); + match download_lambda_zip(&url) { + Ok(data) => { + let mut file = File::create(&zip_path).expect("Failed to create zip file"); + file.write_all(&data).expect("Failed to write zip file"); + println!( + "cargo:warning=Downloaded Lambda zip to {:?} ({} bytes)", + zip_path, + data.len() + ); } - None => { - // Create a placeholder zip for development builds - // This allows compilation to succeed, but deploy() will fail at runtime - // if someone tries to use auto-deploy without a proper binary - println!("cargo:warning=No Lambda binary found, creating placeholder zip"); - println!("cargo:warning=Set QUICKWIT_LAMBDA_BINARY_PATH or build the binary first"); - create_placeholder_zip(&zip_path); + Err(e) => { + panic!("Failed to download Lambda zip: {}", e); } } } #[cfg(feature = "auto-deploy")] -fn create_lambda_zip(binary_path: &std::path::Path, zip_path: &std::path::Path) { - use std::fs::File; - use std::io::Read; - - use zip::ZipWriter; - use zip::write::FileOptions; - - let mut binary_data = Vec::new(); - File::open(binary_path) - .expect("Failed to open Lambda binary") - .read_to_end(&mut binary_data) - .expect("Failed to read Lambda binary"); - - let zip_file = File::create(zip_path).expect("Failed to create zip file"); - let mut zip = ZipWriter::new(zip_file); - - // Lambda requires the binary to be named "bootstrap" with executable permissions - let options = FileOptions::default() - .compression_method(zip::CompressionMethod::Deflated) - .unix_permissions(0o755); - - zip.start_file("bootstrap", options) - .expect("Failed to start zip file entry"); - - std::io::Write::write_all(&mut zip, &binary_data).expect("Failed to write binary to zip"); - - zip.finish().expect("Failed to finalize zip file"); - - println!( - "cargo:warning=Created Lambda zip at {:?} ({} bytes)", - zip_path, - std::fs::metadata(zip_path).unwrap().len() - ); -} - -#[cfg(feature = "auto-deploy")] -fn create_placeholder_zip(zip_path: &std::path::Path) { - use std::fs::File; - - use zip::ZipWriter; - use zip::write::FileOptions; - - let zip_file = File::create(zip_path).expect("Failed to create placeholder zip file"); - let mut zip = ZipWriter::new(zip_file); - - // Create a placeholder script that returns an error - let placeholder_script = r#"#!/bin/sh -echo "ERROR: This is a placeholder Lambda binary." -echo "The auto-deploy feature requires a properly built Lambda binary." -echo "Please build the quickwit-lambda-leaf-search binary for aarch64-unknown-linux-musl" -echo "and set QUICKWIT_LAMBDA_BINARY_PATH environment variable." -exit 1 -"#; - - let options = FileOptions::default() - .compression_method(zip::CompressionMethod::Deflated) - .unix_permissions(0o755); - - zip.start_file("bootstrap", options) - .expect("Failed to start zip file entry"); - - std::io::Write::write_all(&mut zip, placeholder_script.as_bytes()) - .expect("Failed to write placeholder to zip"); - - zip.finish().expect("Failed to finalize zip file"); +fn download_lambda_zip(url: &str) -> Result, Box> { + let response = ureq::get(url).call()?; + // Set limit higher than MAX_LAMBDA_ZIP_SIZE so we can provide a better error message + let data = response + .into_body() + .with_config() + .limit(MAX_LAMBDA_ZIP_SIZE as u64 + 1) // We download one more byte to trigger the panic below. + .read_to_vec()?; + if data.len() > MAX_LAMBDA_ZIP_SIZE { + panic!( + "Lambda zip is too large ({} bytes, max {} bytes).\nAWS Lambda does not support \ + direct upload of binaries larger than 50MB.\nWorkaround: upload the Lambda zip to S3 \ + and deploy from there instead.", + data.len(), + MAX_LAMBDA_ZIP_SIZE + ); + } + Ok(data) } diff --git a/quickwit/quickwit-lambda/src/bin/leaf_search.rs b/quickwit/quickwit-lambda/src/bin/leaf_search.rs index 376e23dbe71..0029a81a9f6 100644 --- a/quickwit/quickwit-lambda/src/bin/leaf_search.rs +++ b/quickwit/quickwit-lambda/src/bin/leaf_search.rs @@ -21,6 +21,12 @@ use quickwit_lambda::{LambdaSearcherContext, LeafSearchPayload, handle_leaf_sear use tracing::info; use tracing_subscriber::EnvFilter; +#[cfg(feature = "auto-deploy")] +compile_error!( + "auto-deploy is forbidden when building the Lambda binary. (The deployer includes the Lambda \ + binary)" +); + #[tokio::main] async fn main() -> Result<(), Error> { // Initialize tracing with JSON output for CloudWatch diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda/src/config.rs index 3f8eda1cca7..72a0f26c493 100644 --- a/quickwit/quickwit-lambda/src/config.rs +++ b/quickwit/quickwit-lambda/src/config.rs @@ -14,8 +14,6 @@ use anyhow::Context as _; use bytesize::ByteSize; -// Re-export LambdaConfig and LambdaDeployConfig from quickwit-config -pub use quickwit_config::{LambdaConfig, LambdaDeployConfig}; /// Configuration for the Lambda handler's SearcherContext. /// These settings are optimized for Lambda's memory constraints. @@ -23,33 +21,30 @@ pub use quickwit_config::{LambdaConfig, LambdaDeployConfig}; pub struct LambdaSearcherConfig { /// Maximum concurrent split searches within a single Lambda invocation. pub max_concurrent_split_searches: usize, - /// Warmup memory budget. pub warmup_memory_budget: ByteSize, } impl LambdaSearcherConfig { pub fn try_from_env() -> anyhow::Result { - let memory_mb: usize = + let lambda_memory_mib: u64 = quickwit_common::get_from_env_opt("AWS_LAMBDA_FUNCTION_MEMORY_SIZE", false) .context("could not get aws lambda function memory size from ENV")?; - Ok(LambdaSearcherConfig::for_memory(memory_mb)) + let lambda_memory = ByteSize::mib(lambda_memory_mib); + Self::for_memory(lambda_memory) } /// Create a Lambda-optimized searcher config based on the allocated memory. - pub fn for_memory(memory_mb: usize) -> Self { + pub fn for_memory(lambda_memory: ByteSize) -> anyhow::Result { // Warmup budget is about half of memory - let warmup_memory_budget = ByteSize::mb((memory_mb / 2) as u64); - - Self { + anyhow::ensure!( + lambda_memory >= ByteSize::gib(1u64), + "lambda memory must be at least 500MB" + ); + let warmup_memory_budget = + ByteSize::b(lambda_memory.as_u64() - ByteSize::mib(500).as_u64()); + Ok(Self { max_concurrent_split_searches: 20, warmup_memory_budget, - } + }) } } - -// impl Default for LambdaSearcherConfig { -// fn default() -> Self { -// // Default to 1024 MB Lambda -// Self::for_memory(1024) -// } -// } diff --git a/quickwit/quickwit-lambda/src/deployer.rs b/quickwit/quickwit-lambda/src/deployer.rs index 4e6a3cd3599..e3e933f4cf8 100644 --- a/quickwit/quickwit-lambda/src/deployer.rs +++ b/quickwit/quickwit-lambda/src/deployer.rs @@ -28,7 +28,6 @@ use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; use quickwit_config::LambdaDeployConfig; use tracing::{debug, info, warn}; -use crate::config::LambdaDeployConfig; use crate::error::{LambdaError, LambdaResult}; /// Embedded Lambda binary (arm64, compressed). @@ -102,8 +101,6 @@ impl LambdaDeployer { } } - - /// Create the Lambda function. /// /// Note: CreateFunction is idempotent - if the function already exists, AWS returns @@ -324,7 +321,6 @@ impl LambdaDeployer { } /// Get function details from AWS. - #[cfg(feature = "auto-deploy")] async fn get_function(&self, name: &str) -> LambdaResult { self.client .get_function() diff --git a/quickwit/quickwit-lambda/src/invoker.rs b/quickwit/quickwit-lambda/src/invoker.rs index 9af909b01b1..50b039211e0 100644 --- a/quickwit/quickwit-lambda/src/invoker.rs +++ b/quickwit/quickwit-lambda/src/invoker.rs @@ -12,29 +12,42 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + use async_trait::async_trait; use aws_sdk_lambda::Client as LambdaClient; use aws_sdk_lambda::primitives::Blob; use aws_sdk_lambda::types::InvocationType; use base64::prelude::*; use prost::Message; +use quickwit_config::LambdaConfig; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{RemoteFunctionInvoker, SearchError}; use tracing::{debug, instrument}; -use crate::config::LambdaConfig; use crate::error::{LambdaError, LambdaResult}; use crate::handler::{LeafSearchPayload, LeafSearchResponsePayload}; +/// Create a Lambda invoker for remote leaf search execution. +/// +/// This creates and validates an AWS Lambda invoker that implements `RemoteFunctionInvoker`. +pub async fn create_lambda_invoker( + config: &LambdaConfig, +) -> LambdaResult> { + let invoker = AwsLambdaInvoker::new(config).await?; + invoker.validate().await?; + Ok(Arc::new(invoker)) +} + /// AWS Lambda implementation of RemoteFunctionInvoker. -pub struct AwsLambdaInvoker { +struct AwsLambdaInvoker { client: LambdaClient, function_name: String, } impl AwsLambdaInvoker { /// Create a new AWS Lambda invoker with the given configuration. - pub async fn new(config: &LambdaConfig) -> LambdaResult { + async fn new(config: &LambdaConfig) -> LambdaResult { let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; let client = LambdaClient::new(&aws_config); @@ -45,10 +58,8 @@ impl AwsLambdaInvoker { } /// Create a new AWS Lambda invoker with a custom client. - pub fn with_client( - client: LambdaClient, - function_name: String, - ) -> Self { + #[cfg(test)] + fn with_client(client: LambdaClient, function_name: String) -> Self { Self { client, function_name, @@ -57,17 +68,12 @@ impl AwsLambdaInvoker { /// Validate that the Lambda function exists and is invocable. /// Uses DryRun invocation type - validates without executing. - pub async fn validate(&self) -> LambdaResult<()> { - let mut request = self + async fn validate(&self) -> LambdaResult<()> { + let request = self .client .invoke() .function_name(&self.function_name) .invocation_type(InvocationType::DryRun); - - if let Some(qualifier) = &self.qualifier { - request = request.qualifier(qualifier); - } - request.send().await.map_err(|e| { LambdaError::Configuration(format!( "Failed to validate Lambda function '{}': {}", @@ -101,17 +107,12 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { ); // Invoke Lambda synchronously (RequestResponse) - let mut invoke_builder = self + let invoke_builder = self .client .invoke() .function_name(&self.function_name) .invocation_type(InvocationType::RequestResponse) .payload(Blob::new(payload_json)); - - if let Some(qualifier) = &self.qualifier { - invoke_builder = invoke_builder.qualifier(qualifier); - } - let response = invoke_builder .send() .await diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda/src/lib.rs index f5c7c8c77b4..c9dbbfa9224 100644 --- a/quickwit/quickwit-lambda/src/lib.rs +++ b/quickwit/quickwit-lambda/src/lib.rs @@ -21,27 +21,19 @@ mod config; mod context; -mod deployer; mod error; mod handler; mod invoker; -pub use config::{LambdaConfig, LambdaSearcherConfig}; pub use context::LambdaSearcherContext; -#[cfg(feature = "auto-deploy")] -mod deployer; -#[cfg(feature = "auto-deploy")] -pub use deployer::LambdaDeployer; - #[cfg(feature = "auto-deploy")] mod deployer; #[cfg(feature = "auto-deploy")] pub use deployer::deploy; - pub use error::{LambdaError, LambdaResult}; pub use handler::{LeafSearchPayload, LeafSearchResponsePayload, handle_leaf_search}; -pub use invoker::AwsLambdaInvoker; +pub use invoker::create_lambda_invoker; /// Deploy is a no-op when auto-deploy feature is not enabled. #[cfg(not(feature = "auto-deploy"))] diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index f455d3b8974..23043ed9869 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -87,7 +87,6 @@ use quickwit_ingest::{ }; use quickwit_jaeger::JaegerService; use quickwit_janitor::{JanitorService, start_janitor_service}; -use quickwit_lambda::{AwsLambdaInvoker, LambdaDeployer}; use quickwit_metastore::{ ControlPlaneMetastore, ListIndexesMetadataResponseExt, MetastoreResolver, }; @@ -107,8 +106,8 @@ use quickwit_proto::metastore::{ use quickwit_proto::search::ReportSplitsRequest; use quickwit_proto::types::NodeId; use quickwit_search::{ - RemoteFunctionInvoker, SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, - SearcherPool, create_search_client_from_channel, start_searcher_service, + SearchJobPlacer, SearchService, SearchServiceClient, SearcherContext, SearcherPool, + create_search_client_from_channel, start_searcher_service, }; use quickwit_storage::{SplitCache, StorageResolver}; use tcp_listener::TcpListenerResolver; @@ -1020,31 +1019,27 @@ async fn setup_searcher( let search_job_placer = SearchJobPlacer::new(searcher_pool.clone()); // Initialize Lambda invoker if enabled - let lambda_invoker: Option> = - if let Some(lambda_config) = &node_config.searcher_config.lambda { - info!("initializing AWS Lambda invoker for leaf search"); - - // Auto-deploy Lambda function if configured - if let Some(deploy_config) = &lambda_config.auto_deploy { - info!("auto-deploying Lambda function"); - quickwit_lambda::deploy(&lambda_config.function_name, deploy_config) - .context("failed to deploy lambda function")?; - } - - let invoker = AwsLambdaInvoker::new(lambda_config) + let lambda_invoker = if let Some(lambda_config) = &node_config.searcher_config.lambda { + info!("initializing AWS Lambda invoker for leaf search"); + + // Auto-deploy Lambda function if configured + if let Some(deploy_config) = &lambda_config.auto_deploy { + info!("auto-deploying Lambda function"); + use anyhow::Context; + quickwit_lambda::deploy(&lambda_config.function_name, deploy_config) .await - .context("failed to initialize AWS Lambda invoker")?; + .context("failed to deploy lambda function")?; + } - // Validate function exists and is accessible - invoker - .validate() - .await - .context("Lambda function validation failed")?; + use anyhow::Context; + let invoker = quickwit_lambda::create_lambda_invoker(lambda_config) + .await + .context("failed to initialize AWS Lambda invoker")?; - Some(Arc::new(invoker)) - } else { - None - }; + Some(invoker) + } else { + None + }; let search_service = start_searcher_service( metastore, From aac7014956f3e335f9f43dfe13d01dd80105cd0e Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Thu, 29 Jan 2026 12:16:03 +0100 Subject: [PATCH 14/22] sever/vclient --- .github/workflows/publish_lambda.yaml | 2 +- LICENSE-3rdparty.csv | 2 + quickwit/Cargo.lock | 29 ++- quickwit/Cargo.toml | 9 +- quickwit/quickwit-cli/Cargo.toml | 3 - quickwit/quickwit-lambda-client/Cargo.toml | 33 +++ .../build.rs | 33 +-- .../src/deployer.rs | 56 +++-- quickwit/quickwit-lambda-client/src/error.rs | 90 +++++++ .../src/invoker.rs | 21 +- .../src/lib.rs | 28 +-- .../Cargo.toml | 29 +-- .../src/bin/leaf_search.rs | 8 +- .../src/config.rs | 0 .../src/context.rs | 0 .../src/error.rs | 22 +- .../src/handler.rs | 0 quickwit/quickwit-lambda-server/src/lib.rs | 28 +++ quickwit/quickwit-lambda/README.md | 232 ------------------ quickwit/quickwit-proto/src/error.rs | 2 +- quickwit/quickwit-search/src/leaf.rs | 2 +- quickwit/quickwit-serve/Cargo.toml | 5 +- quickwit/quickwit-serve/src/lib.rs | 4 +- 23 files changed, 243 insertions(+), 395 deletions(-) create mode 100644 quickwit/quickwit-lambda-client/Cargo.toml rename quickwit/{quickwit-lambda => quickwit-lambda-client}/build.rs (80%) rename quickwit/{quickwit-lambda => quickwit-lambda-client}/src/deployer.rs (89%) create mode 100644 quickwit/quickwit-lambda-client/src/error.rs rename quickwit/{quickwit-lambda => quickwit-lambda-client}/src/invoker.rs (89%) rename quickwit/{quickwit-lambda => quickwit-lambda-client}/src/lib.rs (53%) rename quickwit/{quickwit-lambda => quickwit-lambda-server}/Cargo.toml (64%) rename quickwit/{quickwit-lambda => quickwit-lambda-server}/src/bin/leaf_search.rs (86%) rename quickwit/{quickwit-lambda => quickwit-lambda-server}/src/config.rs (100%) rename quickwit/{quickwit-lambda => quickwit-lambda-server}/src/context.rs (100%) rename quickwit/{quickwit-lambda => quickwit-lambda-server}/src/error.rs (71%) rename quickwit/{quickwit-lambda => quickwit-lambda-server}/src/handler.rs (100%) create mode 100644 quickwit/quickwit-lambda-server/src/lib.rs delete mode 100644 quickwit/quickwit-lambda/README.md diff --git a/.github/workflows/publish_lambda.yaml b/.github/workflows/publish_lambda.yaml index 3700188d734..63f3307e861 100644 --- a/.github/workflows/publish_lambda.yaml +++ b/.github/workflows/publish_lambda.yaml @@ -45,7 +45,7 @@ jobs: echo "QW_COMMIT_TAGS=$(git tag --points-at HEAD | tr '\n' ',')" >> $GITHUB_ENV - name: Build Lambda binary - run: cross build --release --features lambda-release --target aarch64-unknown-linux-gnu -p quickwit-lambda --bin quickwit-aws-lambda-leaf-search + run: cross build --release --features lambda-release --target aarch64-unknown-linux-gnu -p quickwit-lambda-server --bin quickwit-aws-lambda-leaf-search env: QW_COMMIT_DATE: ${{ env.QW_COMMIT_DATE }} QW_COMMIT_HASH: ${{ env.QW_COMMIT_HASH }} diff --git a/LICENSE-3rdparty.csv b/LICENSE-3rdparty.csv index 1a640e966b1..6ace87f3394 100644 --- a/LICENSE-3rdparty.csv +++ b/LICENSE-3rdparty.csv @@ -529,9 +529,11 @@ unicode-width,https://github.com/unicode-rs/unicode-width,MIT OR Apache-2.0,"kwa unit-prefix,https://codeberg.org/commons-rs/unit-prefix,MIT,"Fabio Valentini , Benjamin Sago " unsafe-libyaml,https://github.com/dtolnay/unsafe-libyaml,MIT,David Tolnay untrusted,https://github.com/briansmith/untrusted,ISC,Brian Smith +ureq-proto,https://github.com/algesten/ureq-proto,MIT OR Apache-2.0,Martin Algesten url,https://github.com/servo/rust-url,MIT OR Apache-2.0,The rust-url developers urlencoding,https://github.com/kornelski/rust_urlencoding,MIT,"Kornel , Bertram Truong " username,https://pijul.org/darcs/user,MIT OR Apache-2.0,Pierre-Étienne Meunier +utf-8,https://github.com/SimonSapin/rust-utf8,MIT OR Apache-2.0,Simon Sapin utf8-ranges,https://github.com/BurntSushi/utf8-ranges,Unlicense OR MIT,Andrew Gallant utf8_iter,https://github.com/hsivonen/utf8_iter,Apache-2.0 OR MIT,Henri Sivonen utf8parse,https://github.com/alacritty/vte,Apache-2.0 OR MIT,"Joe Wilm , Christian Duerr " diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 4ce715ba9c9..8134a01eace 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7348,35 +7348,44 @@ dependencies = [ ] [[package]] -name = "quickwit-lambda" +name = "quickwit-lambda-client" version = "0.8.0" dependencies = [ - "anyhow", "async-trait", "aws-config", "aws-sdk-lambda", "base64 0.22.1", - "bytes", + "prost 0.14.1", + "quickwit-config", + "quickwit-lambda-server", + "quickwit-proto", + "quickwit-search", + "serde_json", + "tokio", + "tracing", + "ureq 3.1.4", +] + +[[package]] +name = "quickwit-lambda-server" +version = "0.8.0" +dependencies = [ + "anyhow", + "base64 0.22.1", "bytesize", "lambda_runtime", - "mockall", "openssl", "prost 0.14.1", - "quickwit-aws", "quickwit-common", "quickwit-config", - "quickwit-doc-mapper", "quickwit-proto", "quickwit-search", "quickwit-storage", "serde", "serde_json", - "tempfile", - "thiserror 2.0.17", "tokio", "tracing", "tracing-subscriber", - "ureq 3.1.4", ] [[package]] @@ -7634,7 +7643,7 @@ dependencies = [ "quickwit-ingest", "quickwit-jaeger", "quickwit-janitor", - "quickwit-lambda", + "quickwit-lambda-client", "quickwit-metastore", "quickwit-opentelemetry", "quickwit-proto", diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index ae3670a2bcc..ec87cedacdf 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -19,7 +19,8 @@ members = [ "quickwit-integration-tests", "quickwit-jaeger", "quickwit-janitor", - "quickwit-lambda", + "quickwit-lambda-client", + "quickwit-lambda-server", "quickwit-macros", "quickwit-metastore", @@ -57,7 +58,8 @@ default-members = [ "quickwit-integration-tests", "quickwit-jaeger", "quickwit-janitor", - "quickwit-lambda", + "quickwit-lambda-client", + "quickwit-lambda-server", "quickwit-macros", "quickwit-metastore", "quickwit-opentelemetry", @@ -361,7 +363,8 @@ quickwit-ingest = { path = "quickwit-ingest" } quickwit-integration-tests = { path = "quickwit-integration-tests" } quickwit-jaeger = { path = "quickwit-jaeger" } quickwit-janitor = { path = "quickwit-janitor" } -quickwit-lambda = { path = "quickwit-lambda" } +quickwit-lambda-client = { path = "quickwit-lambda-client" } +quickwit-lambda-server = { path = "quickwit-lambda-server" } quickwit-macros = { path = "quickwit-macros" } quickwit-metastore = { path = "quickwit-metastore" } quickwit-opentelemetry = { path = "quickwit-opentelemetry" } diff --git a/quickwit/quickwit-cli/Cargo.toml b/quickwit/quickwit-cli/Cargo.toml index 6e0e09868e7..c595cb7e90a 100644 --- a/quickwit/quickwit-cli/Cargo.toml +++ b/quickwit/quickwit-cli/Cargo.toml @@ -102,7 +102,6 @@ release-feature-set = [ "quickwit-indexing/pulsar", "quickwit-indexing/sqs", "quickwit-indexing/vrl", - "quickwit-serve/lambda-auto-deploy", "quickwit-storage/azure", "quickwit-storage/gcs", "quickwit-metastore/postgres", @@ -117,7 +116,6 @@ release-feature-vendored-set = [ "quickwit-indexing/sqs", "quickwit-indexing/vrl", "quickwit-indexing/vendored-kafka", - "quickwit-serve/lambda-auto-deploy", "quickwit-storage/azure", "quickwit-storage/gcs", "quickwit-metastore/postgres", @@ -131,7 +129,6 @@ release-macos-feature-vendored-set = [ "quickwit-indexing/sqs", "quickwit-indexing/vrl", "quickwit-indexing/vendored-kafka-macos", - "quickwit-serve/lambda-auto-deploy", "quickwit-storage/azure", "quickwit-storage/gcs", "quickwit-metastore/postgres", diff --git a/quickwit/quickwit-lambda-client/Cargo.toml b/quickwit/quickwit-lambda-client/Cargo.toml new file mode 100644 index 00000000000..bdd0c1bb692 --- /dev/null +++ b/quickwit/quickwit-lambda-client/Cargo.toml @@ -0,0 +1,33 @@ +[package] +name = "quickwit-lambda-client" +description = "AWS Lambda client for Quickwit leaf search invocation and deployment" + +version.workspace = true +edition.workspace = true +homepage.workspace = true +documentation.workspace = true +repository.workspace = true +authors.workspace = true +license.workspace = true + +[dependencies] +async-trait = { workspace = true } +aws-config = { workspace = true } +aws-sdk-lambda = { workspace = true } +base64 = { workspace = true } +prost = { workspace = true } +serde_json = { workspace = true } +tokio = { workspace = true } +tracing = { workspace = true } + +quickwit-config = { workspace = true } +quickwit-lambda-server = { workspace = true } +quickwit-proto = { workspace = true } +quickwit-search = { workspace = true } + +[build-dependencies] +ureq = { workspace = true } + +[features] +default = [] +testsuite = [] diff --git a/quickwit/quickwit-lambda/build.rs b/quickwit/quickwit-lambda-client/build.rs similarity index 80% rename from quickwit/quickwit-lambda/build.rs rename to quickwit/quickwit-lambda-client/build.rs index 44bfc404eec..f18ca019aea 100644 --- a/quickwit/quickwit-lambda/build.rs +++ b/quickwit/quickwit-lambda-client/build.rs @@ -12,40 +12,28 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! Build script for quickwit-lambda. +//! Build script for quickwit-lambda-client. //! -//! When the `auto-deploy` feature is enabled, this script: -//! 1. Downloads the pre-built Lambda zip from a GitHub release -//! 2. Places the zip in OUT_DIR for embedding via include_bytes! +//! This script downloads the pre-built Lambda zip from a GitHub release +//! and places it in OUT_DIR for embedding via include_bytes! //! //! The Lambda binary is built separately in CI and published as a GitHub release. -fn main() { - #[cfg(feature = "auto-deploy")] - auto_deploy_build(); - - #[cfg(not(feature = "auto-deploy"))] - println!("cargo:rerun-if-changed=build.rs"); -} +use std::env; +use std::fs::File; +use std::io::Write; +use std::path::PathBuf; /// URL to download the pre-built Lambda zip from GitHub releases. /// This should be updated when a new Lambda binary is released. -#[cfg(feature = "auto-deploy")] const LAMBDA_ZIP_URL: &str = "https://github.com/quickwit-oss/quickwit/releases/download/lambda-506751fb/quickwit-aws-lambda--aarch64.zip"; /// AWS Lambda direct upload limit is 50MB. /// Larger artifacts must be uploaded via S3. -#[cfg(feature = "auto-deploy")] const MAX_LAMBDA_ZIP_SIZE: usize = 50 * 1024 * 1024; -#[cfg(feature = "auto-deploy")] -fn auto_deploy_build() { - use std::env; - use std::fs::File; - use std::io::Write; - use std::path::PathBuf; - +fn main() { println!("cargo:rerun-if-changed=build.rs"); println!("cargo:rerun-if-env-changed=QUICKWIT_LAMBDA_ZIP_URL"); @@ -73,11 +61,10 @@ fn auto_deploy_build() { } } -#[cfg(feature = "auto-deploy")] fn download_lambda_zip(url: &str) -> Result, Box> { - let response = ureq::get(url).call()?; + let response = ureq::get(url).call(); // Set limit higher than MAX_LAMBDA_ZIP_SIZE so we can provide a better error message - let data = response + let data = response? .into_body() .with_config() .limit(MAX_LAMBDA_ZIP_SIZE as u64 + 1) // We download one more byte to trigger the panic below. diff --git a/quickwit/quickwit-lambda/src/deployer.rs b/quickwit/quickwit-lambda-client/src/deployer.rs similarity index 89% rename from quickwit/quickwit-lambda/src/deployer.rs rename to quickwit/quickwit-lambda-client/src/deployer.rs index e3e933f4cf8..199ab7969a1 100644 --- a/quickwit/quickwit-lambda/src/deployer.rs +++ b/quickwit/quickwit-lambda-client/src/deployer.rs @@ -28,10 +28,10 @@ use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; use quickwit_config::LambdaDeployConfig; use tracing::{debug, info, warn}; -use crate::error::{LambdaError, LambdaResult}; +use crate::error::{LambdaClientError, LambdaClientResult}; /// Embedded Lambda binary (arm64, compressed). -/// This is included at compile time when the `auto-deploy` feature is enabled. +/// This is included at compile time. const LAMBDA_BINARY: &[u8] = include_bytes!(concat!(env!("OUT_DIR"), "/lambda_bootstrap.zip")); /// Version tag key used to track deployed Quickwit version. @@ -50,17 +50,12 @@ pub struct LambdaDeployer { impl LambdaDeployer { /// Create a new Lambda deployer using default AWS configuration. - pub async fn new() -> LambdaResult { + pub async fn new() -> LambdaClientResult { let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; let client = LambdaClient::new(&aws_config); Ok(Self { client }) } - /// Create a new Lambda deployer with a custom client. - pub fn with_client(client: LambdaClient) -> Self { - Self { client } - } - /// Deploy or update the Lambda function. /// /// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. @@ -69,7 +64,7 @@ impl LambdaDeployer { &self, function_name: &str, deploy_config: &LambdaDeployConfig, - ) -> LambdaResult { + ) -> LambdaClientResult { let role_arn = &deploy_config.execution_role_arn; match self.get_function(function_name).await { @@ -77,14 +72,14 @@ impl LambdaDeployer { self.update_function_if_needed(function_name, &existing, deploy_config) .await } - Err(LambdaError::NotFound(_)) => { + Err(LambdaClientError::NotFound(_)) => { // Function doesn't exist, try to create it match self .create_function(function_name, role_arn, deploy_config) .await { Ok(arn) => Ok(arn), - Err(LambdaError::ResourceConflict) => { + Err(LambdaClientError::ResourceConflict) => { // Another node created the function concurrently, update instead info!( function_name = %function_name, @@ -111,7 +106,7 @@ impl LambdaDeployer { name: &str, role: &str, config: &LambdaDeployConfig, - ) -> LambdaResult { + ) -> LambdaClientResult { info!( function_name = %name, role = %role, @@ -152,7 +147,9 @@ impl LambdaDeployer { Ok(output) => { let arn = output .function_arn() - .ok_or_else(|| LambdaError::Deployment("no function ARN returned".into()))? + .ok_or_else(|| { + LambdaClientError::Deployment("no function ARN returned".into()) + })? .to_string(); info!(function_arn = %arn, "Lambda function created successfully"); Ok(arn) @@ -160,9 +157,9 @@ impl LambdaDeployer { Err(SdkError::ServiceError(err)) if matches!(err.err(), CreateFunctionError::ResourceConflictException(_)) => { - Err(LambdaError::ResourceConflict) + Err(LambdaClientError::ResourceConflict) } - Err(e) => Err(LambdaError::Deployment(format!( + Err(e) => Err(LambdaClientError::Deployment(format!( "failed to create function: {}", e ))), @@ -178,11 +175,13 @@ impl LambdaDeployer { name: &str, existing: &GetFunctionOutput, config: &LambdaDeployConfig, - ) -> LambdaResult { + ) -> LambdaClientResult { let function_arn = existing .configuration() .and_then(|c| c.function_arn()) - .ok_or_else(|| LambdaError::Deployment("no function ARN in existing config".into()))? + .ok_or_else(|| { + LambdaClientError::Deployment("no function ARN in existing config".into()) + })? .to_string(); if !self.needs_update(existing) { @@ -208,7 +207,7 @@ impl LambdaDeployer { .send() .await .map_err(|e| { - LambdaError::Deployment(format!("failed to update function code: {}", e)) + LambdaClientError::Deployment(format!("failed to update function code: {}", e)) })?; // Wait for the update to complete before updating configuration @@ -229,7 +228,10 @@ impl LambdaDeployer { .send() .await .map_err(|e| { - LambdaError::Deployment(format!("failed to update function configuration: {}", e)) + LambdaClientError::Deployment(format!( + "failed to update function configuration: {}", + e + )) })?; // Wait for config update to complete before updating tags @@ -243,7 +245,7 @@ impl LambdaDeployer { .send() .await .map_err(|e| { - LambdaError::Deployment(format!("failed to update function tags: {}", e)) + LambdaClientError::Deployment(format!("failed to update function tags: {}", e)) })?; info!(function_arn = %function_arn, "Lambda function updated successfully"); @@ -251,7 +253,7 @@ impl LambdaDeployer { } /// Wait for function update to complete. - async fn wait_for_update_complete(&self, name: &str) -> LambdaResult<()> { + async fn wait_for_update_complete(&self, name: &str) -> LambdaClientResult<()> { // Poll until the function state is Active and LastUpdateStatus is Successful for _ in 0..60 { let output = self.get_function(name).await?; @@ -268,7 +270,7 @@ impl LambdaDeployer { let reason = config .last_update_status_reason() .unwrap_or("unknown reason"); - return Err(LambdaError::Deployment(format!( + return Err(LambdaClientError::Deployment(format!( "function update failed: {}", reason ))); @@ -285,7 +287,7 @@ impl LambdaDeployer { } } } - Err(LambdaError::Deployment( + Err(LambdaClientError::Deployment( "timeout waiting for function update to complete".into(), )) } @@ -321,7 +323,7 @@ impl LambdaDeployer { } /// Get function details from AWS. - async fn get_function(&self, name: &str) -> LambdaResult { + async fn get_function(&self, name: &str) -> LambdaClientResult { self.client .get_function() .function_name(name) @@ -332,9 +334,9 @@ impl LambdaDeployer { if e.to_string().contains("ResourceNotFoundException") || e.to_string().contains("Function not found") { - LambdaError::NotFound(name.to_string()) + LambdaClientError::NotFound(name.to_string()) } else { - LambdaError::Deployment(format!("failed to get function: {}", e)) + LambdaClientError::Deployment(format!("failed to get function: {}", e)) } }) } @@ -364,7 +366,7 @@ impl LambdaDeployer { pub async fn deploy( function_name: &str, deploy_config: &LambdaDeployConfig, -) -> LambdaResult { +) -> LambdaClientResult { let lambda_deployer = LambdaDeployer::new().await?; let lambda_arn = lambda_deployer.deploy(function_name, deploy_config).await?; info!("successfully deployed lambda function `{}`", lambda_arn); diff --git a/quickwit/quickwit-lambda-client/src/error.rs b/quickwit/quickwit-lambda-client/src/error.rs new file mode 100644 index 00000000000..eef384166e5 --- /dev/null +++ b/quickwit/quickwit-lambda-client/src/error.rs @@ -0,0 +1,90 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt; + +use quickwit_search::SearchError; + +/// Result type for Lambda client operations. +pub type LambdaClientResult = Result; + +/// Errors that can occur during Lambda client operations. +#[derive(Debug)] +pub enum LambdaClientError { + /// Error during Lambda invocation. + Invocation(String), + /// Error serializing/deserializing protobuf. + Serialization(String), + /// Lambda function returned an error. + FunctionError(String), + /// Configuration error. + Configuration(String), + /// Internal error. + Internal(String), + /// Resource conflict (e.g., function already exists during concurrent create). + ResourceConflict, + /// Error during Lambda function deployment. + Deployment(String), + /// Lambda function not found. + NotFound(String), +} + +impl fmt::Display for LambdaClientError { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + LambdaClientError::Invocation(msg) => write!(f, "Lambda invocation error: {}", msg), + LambdaClientError::Serialization(msg) => write!(f, "Serialization error: {}", msg), + LambdaClientError::FunctionError(msg) => write!(f, "Lambda function error: {}", msg), + LambdaClientError::Configuration(msg) => write!(f, "Configuration error: {}", msg), + LambdaClientError::Internal(msg) => write!(f, "Internal error: {}", msg), + LambdaClientError::ResourceConflict => { + write!(f, "Resource conflict: function already exists") + } + LambdaClientError::Deployment(msg) => write!(f, "Deployment error: {}", msg), + LambdaClientError::NotFound(name) => write!(f, "Lambda function not found: {}", name), + } + } +} + +impl std::error::Error for LambdaClientError {} + +impl From for LambdaClientError { + fn from(err: prost::DecodeError) -> Self { + LambdaClientError::Serialization(format!("Protobuf decode error: {}", err)) + } +} + +impl From for LambdaClientError { + fn from(err: prost::EncodeError) -> Self { + LambdaClientError::Serialization(format!("Protobuf encode error: {}", err)) + } +} + +impl From for LambdaClientError { + fn from(err: base64::DecodeError) -> Self { + LambdaClientError::Serialization(format!("Base64 decode error: {}", err)) + } +} + +impl From for LambdaClientError { + fn from(err: serde_json::Error) -> Self { + LambdaClientError::Serialization(format!("JSON error: {}", err)) + } +} + +impl From for SearchError { + fn from(err: LambdaClientError) -> Self { + SearchError::Internal(err.to_string()) + } +} diff --git a/quickwit/quickwit-lambda/src/invoker.rs b/quickwit/quickwit-lambda-client/src/invoker.rs similarity index 89% rename from quickwit/quickwit-lambda/src/invoker.rs rename to quickwit/quickwit-lambda-client/src/invoker.rs index 50b039211e0..7837e2456d4 100644 --- a/quickwit/quickwit-lambda/src/invoker.rs +++ b/quickwit/quickwit-lambda-client/src/invoker.rs @@ -21,19 +21,19 @@ use aws_sdk_lambda::types::InvocationType; use base64::prelude::*; use prost::Message; use quickwit_config::LambdaConfig; +use quickwit_lambda_server::{LeafSearchPayload, LeafSearchResponsePayload}; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{RemoteFunctionInvoker, SearchError}; use tracing::{debug, instrument}; -use crate::error::{LambdaError, LambdaResult}; -use crate::handler::{LeafSearchPayload, LeafSearchResponsePayload}; +use crate::error::{LambdaClientError, LambdaClientResult}; /// Create a Lambda invoker for remote leaf search execution. /// /// This creates and validates an AWS Lambda invoker that implements `RemoteFunctionInvoker`. pub async fn create_lambda_invoker( config: &LambdaConfig, -) -> LambdaResult> { +) -> LambdaClientResult> { let invoker = AwsLambdaInvoker::new(config).await?; invoker.validate().await?; Ok(Arc::new(invoker)) @@ -47,7 +47,7 @@ struct AwsLambdaInvoker { impl AwsLambdaInvoker { /// Create a new AWS Lambda invoker with the given configuration. - async fn new(config: &LambdaConfig) -> LambdaResult { + async fn new(config: &LambdaConfig) -> LambdaClientResult { let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; let client = LambdaClient::new(&aws_config); @@ -57,25 +57,16 @@ impl AwsLambdaInvoker { }) } - /// Create a new AWS Lambda invoker with a custom client. - #[cfg(test)] - fn with_client(client: LambdaClient, function_name: String) -> Self { - Self { - client, - function_name, - } - } - /// Validate that the Lambda function exists and is invocable. /// Uses DryRun invocation type - validates without executing. - async fn validate(&self) -> LambdaResult<()> { + async fn validate(&self) -> LambdaClientResult<()> { let request = self .client .invoke() .function_name(&self.function_name) .invocation_type(InvocationType::DryRun); request.send().await.map_err(|e| { - LambdaError::Configuration(format!( + LambdaClientError::Configuration(format!( "Failed to validate Lambda function '{}': {}", self.function_name, e )) diff --git a/quickwit/quickwit-lambda/src/lib.rs b/quickwit/quickwit-lambda-client/src/lib.rs similarity index 53% rename from quickwit/quickwit-lambda/src/lib.rs rename to quickwit/quickwit-lambda-client/src/lib.rs index c9dbbfa9224..b404200ca25 100644 --- a/quickwit/quickwit-lambda/src/lib.rs +++ b/quickwit/quickwit-lambda-client/src/lib.rs @@ -12,36 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -//! AWS Lambda support for Quickwit leaf search operations. +//! AWS Lambda client for Quickwit leaf search operations. //! //! This crate provides: -//! - A Lambda handler that executes leaf search requests //! - An AWS Lambda implementation of the `RemoteFunctionInvoker` trait //! - Auto-deployment functionality for Lambda functions -mod config; -mod context; +mod deployer; mod error; -mod handler; mod invoker; -pub use context::LambdaSearcherContext; - -#[cfg(feature = "auto-deploy")] -mod deployer; -#[cfg(feature = "auto-deploy")] pub use deployer::deploy; -pub use error::{LambdaError, LambdaResult}; -pub use handler::{LeafSearchPayload, LeafSearchResponsePayload, handle_leaf_search}; +pub use error::{LambdaClientError, LambdaClientResult}; pub use invoker::create_lambda_invoker; - -/// Deploy is a no-op when auto-deploy feature is not enabled. -#[cfg(not(feature = "auto-deploy"))] -pub async fn deploy( - _function_name: &str, - _deploy_config: &quickwit_config::LambdaDeployConfig, -) -> LambdaResult { - Err(LambdaError::Configuration( - "auto-deploy feature is not enabled at compile time".into(), - )) -} +// Re-export payload types from server crate for convenience +pub use quickwit_lambda_server::{LeafSearchPayload, LeafSearchResponsePayload}; diff --git a/quickwit/quickwit-lambda/Cargo.toml b/quickwit/quickwit-lambda-server/Cargo.toml similarity index 64% rename from quickwit/quickwit-lambda/Cargo.toml rename to quickwit/quickwit-lambda-server/Cargo.toml index 988202d8502..53e3725131e 100644 --- a/quickwit/quickwit-lambda/Cargo.toml +++ b/quickwit/quickwit-lambda-server/Cargo.toml @@ -1,6 +1,6 @@ [package] -name = "quickwit-lambda" -description = "AWS Lambda support for Quickwit leaf search" +name = "quickwit-lambda-server" +description = "AWS Lambda handler for Quickwit leaf search" version.workspace = true edition.workspace = true @@ -10,54 +10,39 @@ repository.workspace = true authors.workspace = true license.workspace = true +[package.metadata.cargo-machete] +# Its here even though it is not useful, in order to enable its "vendor" feature, +# allowing the cross-build. +ignored = ["openssl"] + [dependencies] anyhow = { workspace = true } -async-trait = { workspace = true } base64 = { workspace = true } -bytes = { workspace = true } bytesize = { workspace = true } lambda_runtime = { workspace = true } prost = { workspace = true } serde = { workspace = true } serde_json = { workspace = true } -thiserror = { workspace = true } tokio = { workspace = true } tracing = { workspace = true } tracing-subscriber = { workspace = true, features = ["env-filter", "json"] } -aws-config = { workspace = true } -aws-sdk-lambda = { workspace = true } - openssl = { workspace = true, optional = true } -quickwit-aws = { workspace = true } quickwit-common = { workspace = true } quickwit-config = { workspace = true } -quickwit-doc-mapper = { workspace = true } quickwit-proto = { workspace = true } quickwit-search = { workspace = true } quickwit-storage = { workspace = true } -[dev-dependencies] -mockall = { workspace = true } -tempfile = { workspace = true } - [[bin]] name = "quickwit-aws-lambda-leaf-search" path = "src/bin/leaf_search.rs" -[build-dependencies] -# Build dependencies for auto-deploy feature. -ureq = { workspace = true, optional = true } - [features] default = [] testsuite = [] -# Enable automatic Lambda function deployment at startup. -# When enabled, the Lambda binary is embedded at compile time. -auto-deploy = ["dep:ureq"] - # Keep this in sync with quickwit-cli! lambda-release = [ # The vendored OpenSSL will be compiled from source during the diff --git a/quickwit/quickwit-lambda/src/bin/leaf_search.rs b/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs similarity index 86% rename from quickwit/quickwit-lambda/src/bin/leaf_search.rs rename to quickwit/quickwit-lambda-server/src/bin/leaf_search.rs index 0029a81a9f6..2cf4419259e 100644 --- a/quickwit/quickwit-lambda/src/bin/leaf_search.rs +++ b/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs @@ -17,16 +17,10 @@ use std::sync::Arc; use lambda_runtime::{Error, LambdaEvent, service_fn}; -use quickwit_lambda::{LambdaSearcherContext, LeafSearchPayload, handle_leaf_search}; +use quickwit_lambda_server::{LambdaSearcherContext, LeafSearchPayload, handle_leaf_search}; use tracing::info; use tracing_subscriber::EnvFilter; -#[cfg(feature = "auto-deploy")] -compile_error!( - "auto-deploy is forbidden when building the Lambda binary. (The deployer includes the Lambda \ - binary)" -); - #[tokio::main] async fn main() -> Result<(), Error> { // Initialize tracing with JSON output for CloudWatch diff --git a/quickwit/quickwit-lambda/src/config.rs b/quickwit/quickwit-lambda-server/src/config.rs similarity index 100% rename from quickwit/quickwit-lambda/src/config.rs rename to quickwit/quickwit-lambda-server/src/config.rs diff --git a/quickwit/quickwit-lambda/src/context.rs b/quickwit/quickwit-lambda-server/src/context.rs similarity index 100% rename from quickwit/quickwit-lambda/src/context.rs rename to quickwit/quickwit-lambda-server/src/context.rs diff --git a/quickwit/quickwit-lambda/src/error.rs b/quickwit/quickwit-lambda-server/src/error.rs similarity index 71% rename from quickwit/quickwit-lambda/src/error.rs rename to quickwit/quickwit-lambda-server/src/error.rs index 5c33f162942..bcc5d78a1de 100644 --- a/quickwit/quickwit-lambda/src/error.rs +++ b/quickwit/quickwit-lambda-server/src/error.rs @@ -19,43 +19,23 @@ use quickwit_search::SearchError; /// Result type for Lambda operations. pub type LambdaResult = Result; -/// Errors that can occur during Lambda operations. +/// Errors that can occur during Lambda handler operations. #[derive(Debug)] pub enum LambdaError { - /// Error during Lambda invocation. - Invocation(String), /// Error serializing/deserializing protobuf. Serialization(String), /// Error from the search operation. Search(SearchError), - /// Lambda function returned an error. - FunctionError(String), - /// Configuration error. - Configuration(String), /// Internal error. Internal(String), - /// Resource conflict (e.g., function already exists during concurrent create). - ResourceConflict, - /// Error during Lambda function deployment. - Deployment(String), - /// Lambda function not found. - NotFound(String), } impl fmt::Display for LambdaError { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { - LambdaError::Invocation(msg) => write!(f, "Lambda invocation error: {}", msg), LambdaError::Serialization(msg) => write!(f, "Serialization error: {}", msg), LambdaError::Search(err) => write!(f, "Search error: {}", err), - LambdaError::FunctionError(msg) => write!(f, "Lambda function error: {}", msg), - LambdaError::Configuration(msg) => write!(f, "Configuration error: {}", msg), LambdaError::Internal(msg) => write!(f, "Internal error: {}", msg), - LambdaError::ResourceConflict => { - write!(f, "Resource conflict: function already exists") - } - LambdaError::Deployment(msg) => write!(f, "Deployment error: {}", msg), - LambdaError::NotFound(name) => write!(f, "Lambda function not found: {}", name), } } } diff --git a/quickwit/quickwit-lambda/src/handler.rs b/quickwit/quickwit-lambda-server/src/handler.rs similarity index 100% rename from quickwit/quickwit-lambda/src/handler.rs rename to quickwit/quickwit-lambda-server/src/handler.rs diff --git a/quickwit/quickwit-lambda-server/src/lib.rs b/quickwit/quickwit-lambda-server/src/lib.rs new file mode 100644 index 00000000000..d67bc6085d4 --- /dev/null +++ b/quickwit/quickwit-lambda-server/src/lib.rs @@ -0,0 +1,28 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! AWS Lambda handler for Quickwit leaf search operations. +//! +//! This crate provides the Lambda handler that executes leaf search requests. +//! It is designed to be deployed as an AWS Lambda function. + +mod config; +mod context; +mod error; +mod handler; + +pub use config::LambdaSearcherConfig; +pub use context::LambdaSearcherContext; +pub use error::{LambdaError, LambdaResult}; +pub use handler::{LeafSearchPayload, LeafSearchResponsePayload, handle_leaf_search}; diff --git a/quickwit/quickwit-lambda/README.md b/quickwit/quickwit-lambda/README.md deleted file mode 100644 index 830c9cd10e1..00000000000 --- a/quickwit/quickwit-lambda/README.md +++ /dev/null @@ -1,232 +0,0 @@ -# Quickwit Lambda - -This crate provides AWS Lambda support for Quickwit's leaf search operations, enabling serverless execution of search queries across splits. - -## Overview - -Quickwit can offload leaf search operations to AWS Lambda functions. This is useful for: -- Scaling search capacity elastically -- Reducing infrastructure costs for sporadic workloads -- Isolating search execution from the main Quickwit cluster - -## Features - -### Lambda Invoker - -The `RemoteFunctionInvoker` trait is meant to allow implementing a custom invoker for other cloud providers, -but at the moment only AWS is supported. - -### Auto-Deploy (Optional) - -When the `auto-deploy` feature is enabled, Quickwit can automatically create or update the Lambda function at startup. This eliminates the need to manually deploy the Lambda function. - -## Configuration - -Add the following to your Quickwit configuration. Lambda is enabled by including the `lambda` section under `searcher`: - -```yaml -searcher: - lambda: - # Lambda function settings - function_name: "quickwit-lambda-search" - max_splits_per_invocation: 10 - - # Auto-deploy settings (requires auto-deploy feature) - auto_deploy: - execution_role_arn: "arn:aws:iam::123456789:role/quickwit-lambda-role" - memory_size: 5GiB - invocation_timeout_secs: 30 -``` - -### Configuration Options - -#### LambdaConfig (under `searcher.lambda`) - -| Option | Type | Default | Description | -|--------|------|---------|-------------| -| `function_name` | string | `"quickwit-lambda-search"` | Lambda function name or ARN | -| `function_qualifier` | string | `null` | Optional alias or version qualifier | -| `max_splits_per_invocation` | int | `10` | Maximum splits per Lambda invocation | -| `auto_deploy` | object | `null` | Auto-deploy configuration (see below) | - -#### LambdaDeployConfig (under `searcher.lambda.auto_deploy`) - -| Option | Type | Default | Description | -|--------|------|---------|-------------| -| `execution_role_arn` | string | required | IAM role ARN for the Lambda function | -| `memory_size` | ByteSize | `5GiB` | Lambda memory allocation | -| `invocation_timeout_secs` | int | `30` | Timeout for Lambda invocations in seconds | - -## Auto-Deploy Feature - -The auto-deploy feature allows Quickwit to automatically manage the Lambda function lifecycle. - -### Build-Time Binary Packaging - -When building Quickwit with the `auto-deploy` feature enabled, the build script (`build.rs`) packages the Lambda binary into a zip file that gets embedded in the Quickwit binary: - -```bash -cargo build --release -p quickwit-lambda --features auto-deploy -``` - -The build script looks for the Lambda binary in the following locations (in order): - -1. `QUICKWIT_LAMBDA_BINARY_PATH` environment variable -2. `target/aarch64-unknown-linux-musl/release/quickwit-lambda-leaf-search` -3. `target/lambda/quickwit-lambda-leaf-search/bootstrap` -4. `lambda_bootstrap` in the workspace root - -If no binary is found, a placeholder is created. This allows development builds to succeed, but the placeholder will fail if actually deployed. - -**For production builds**, you must first cross-compile the Lambda binary for ARM64 Linux: - -```bash -# Install the target -rustup target add aarch64-unknown-linux-musl - -# Build the Lambda binary -cargo build --release --target aarch64-unknown-linux-musl -p quickwit-lambda --bin quickwit-lambda-leaf-search - -# Then build Quickwit with auto-deploy -cargo build --release --features auto-deploy -``` - -### Startup Behavior - -When `auto_deploy` is configured, the following happens at Quickwit startup: - -1. **Function Check**: Quickwit checks if the Lambda function exists -2. **Create or Update**: - - If the function doesn't exist, it creates it - - If the function exists but has a different version tag, it updates the code and configuration - - If the function exists with the same version, no action is taken -3. **Validation**: After deployment, Quickwit validates the function is invocable - -**Important**: If deployment fails for any reason, Quickwit startup fails immediately. This fail-fast behavior ensures you don't run a Quickwit cluster that expects Lambda execution but can't actually invoke the function. - -### Multi-Node Coordination - -The auto-deploy feature is safe for concurrent execution from multiple Quickwit nodes: - -- AWS Lambda's `CreateFunction` API is idempotent - if multiple nodes try to create the same function simultaneously, one succeeds and others receive a `ResourceConflictException` -- When a conflict is detected, the node falls back to updating the existing function -- Version tracking via tags ensures updates only happen when necessary - -### Version Management - -Quickwit tracks the deployed Lambda version using a `quickwit_version` tag on the function. On startup: - -- If the tag matches the current Quickwit version, no update is performed -- If the tag differs or is missing, the function is updated - -This means upgrading Quickwit automatically updates the Lambda function to match. - -## IAM Permissions - -### For the Quickwit Process (Auto-Deploy) - -The Quickwit process needs these permissions to auto-deploy: - -```json -{ - "Version": "2012-10-17", - "Statement": [ - { - "Effect": "Allow", - "Action": [ - "lambda:CreateFunction", - "lambda:UpdateFunctionCode", - "lambda:UpdateFunctionConfiguration", - "lambda:GetFunction", - "lambda:TagResource" - ], - "Resource": "arn:aws:lambda:*:*:function:quickwit-*" - }, - { - "Effect": "Allow", - "Action": "iam:PassRole", - "Resource": "arn:aws:iam::*:role/quickwit-lambda-*" - } - ] -} -``` - -### For Invoking the Lambda - -The Quickwit process needs permission to invoke the function: - -```json -{ - "Version": "2012-10-17", - "Statement": [ - { - "Effect": "Allow", - "Action": "lambda:InvokeFunction", - "Resource": "arn:aws:lambda:*:*:function:quickwit-*" - } - ] -} -``` - -### Lambda Execution Role - -The Lambda function itself needs an execution role with: - -```json -{ - "Version": "2012-10-17", - "Statement": [ - { - "Effect": "Allow", - "Action": [ - "logs:CreateLogGroup", - "logs:CreateLogStream", - "logs:PutLogEvents" - ], - "Resource": "arn:aws:logs:*:*:*" - }, - { - "Effect": "Allow", - "Action": [ - "s3:GetObject", - "s3:ListBucket" - ], - "Resource": [ - "arn:aws:s3:::your-index-bucket", - "arn:aws:s3:::your-index-bucket/*" - ] - } - ] -} -``` - -## Architecture - -The Lambda function: -- Runs on ARM64 (Graviton2) for cost efficiency -- Uses the `provided.al2023` runtime -- Receives protobuf-encoded `LeafSearchRequest` payloads (base64 in JSON) -- Returns protobuf-encoded `LeafSearchResponse` payloads (base64 in JSON) - -## Troubleshooting - -### Startup Failures - -If Quickwit fails to start with Lambda enabled: - -1. **"execution_role_arn required"**: The `auto_deploy` section is present but the `execution_role_arn` field is empty -2. **"failed to create function"**: Check IAM permissions for the Quickwit process -3. **"Lambda function validation failed"**: The function exists but isn't invocable - check the function's execution role - -### Runtime Issues - -1. **Timeouts**: Increase `invocation_timeout_secs` -2. **Memory errors**: Increase `memory_size` -3. **Throttling**: AWS Lambda has concurrency limits - check your account limits - -### Build Issues - -If you see "No Lambda binary found, creating placeholder zip": -- This is expected for development builds without cross-compilation -- For production, ensure the Lambda binary is built for `aarch64-unknown-linux-musl` -- Set `QUICKWIT_LAMBDA_BINARY_PATH` to point to your pre-built binary diff --git a/quickwit/quickwit-proto/src/error.rs b/quickwit/quickwit-proto/src/error.rs index b03536d0283..aa3905135a1 100644 --- a/quickwit/quickwit-proto/src/error.rs +++ b/quickwit/quickwit-proto/src/error.rs @@ -97,7 +97,7 @@ where E: ServiceError } /// A trait for encoding/decoding service errors to/from gRPC statuses. Errors are stored in JSON -/// in the gRPC header [`QW_ERROR_HEADER_NAME`]. This allows for propagating them transparently +/// in the gRPC header `qw-error-bin`. This allows for propagating them transparently /// between clients and servers over the network without being semantically limited to a status code /// and a message. However, it also means that modifying the serialization format of existing errors /// or introducing new ones is not backward compatible. diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 514277b2148..dd2dcb1e67c 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -1355,7 +1355,7 @@ fn disable_search_request_hits(search_request: &mut SearchRequest) { /// Searches multiple splits for a specific index and a single doc mapping /// /// The leaf search collects all kind of information, and returns a set of -/// [PartialHit](quickwit_proto::search::PartialHit) candidates. The root will be in +/// [PartialHit] candidates. The root will be in /// charge to consolidate, identify the actual final top hits to display, and /// fetch the actual documents to convert the partial hits into actual Hits. pub async fn single_doc_mapping_leaf_search( diff --git a/quickwit/quickwit-serve/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index 07ac742ac19..16f8a36002d 100644 --- a/quickwit/quickwit-serve/Cargo.toml +++ b/quickwit/quickwit-serve/Cargo.toml @@ -72,7 +72,7 @@ quickwit-opentelemetry = { workspace = true } quickwit-proto = { workspace = true } quickwit-query = { workspace = true } quickwit-search = { workspace = true } -quickwit-lambda = { workspace = true } +quickwit-lambda-client = { workspace = true } quickwit-storage = { workspace = true } quickwit-telemetry = { workspace = true } @@ -110,9 +110,6 @@ pprof = [ jemalloc-profiled = [ "quickwit-common/jemalloc-profiled" ] -lambda-auto-deploy = [ - "quickwit-lambda/auto-deploy" -] testsuite = [] sqs-for-tests = [ "quickwit-indexing/sqs", diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 23043ed9869..be0d7f64397 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1026,13 +1026,13 @@ async fn setup_searcher( if let Some(deploy_config) = &lambda_config.auto_deploy { info!("auto-deploying Lambda function"); use anyhow::Context; - quickwit_lambda::deploy(&lambda_config.function_name, deploy_config) + quickwit_lambda_client::deploy(&lambda_config.function_name, deploy_config) .await .context("failed to deploy lambda function")?; } use anyhow::Context; - let invoker = quickwit_lambda::create_lambda_invoker(lambda_config) + let invoker = quickwit_lambda_client::create_lambda_invoker(lambda_config) .await .context("failed to initialize AWS Lambda invoker")?; From 7a4688b1ff21ed25354ef3b16d001cdb23f07579 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Thu, 29 Jan 2026 21:06:21 +0100 Subject: [PATCH 15/22] bnlop --- .github/workflows/publish_lambda.yaml | 7 ++++++- quickwit/quickwit-common/src/rate_limiter.rs | 1 + quickwit/quickwit-lambda-server/README.md | 13 +++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) create mode 100644 quickwit/quickwit-lambda-server/README.md diff --git a/.github/workflows/publish_lambda.yaml b/.github/workflows/publish_lambda.yaml index 63f3307e861..2275e9241fb 100644 --- a/.github/workflows/publish_lambda.yaml +++ b/.github/workflows/publish_lambda.yaml @@ -1,4 +1,9 @@ -name: Build Lambda binary +# This workflow creates a new release for a quickwit search aws lambda. +# The artifact is a zip file containing a binary for ARM 64, +# ready to be deployed by the deployer. +# +# See quickwit-lambda/README.md +name: Release Lambda binary on: push: diff --git a/quickwit/quickwit-common/src/rate_limiter.rs b/quickwit/quickwit-common/src/rate_limiter.rs index 5b7c2e4e92a..739ce8428c0 100644 --- a/quickwit/quickwit-common/src/rate_limiter.rs +++ b/quickwit/quickwit-common/src/rate_limiter.rs @@ -33,6 +33,7 @@ pub struct RateLimiterSettings { pub refill_period: Duration, } +#[cfg(any(test, feature = "testsuite"))] impl Default for RateLimiterSettings { fn default() -> Self { // 10 MB burst limit. diff --git a/quickwit/quickwit-lambda-server/README.md b/quickwit/quickwit-lambda-server/README.md new file mode 100644 index 00000000000..07c8190be41 --- /dev/null +++ b/quickwit/quickwit-lambda-server/README.md @@ -0,0 +1,13 @@ +# Lambda search + +Quickwit makes it possible to run leaf search on lambdas. + +In order to make it possible to update quickwit AND the function you +can have Quickwit itself can deploy/create the function. + +For that purpose Quickwit client's deployer embeds the lambda function zip file +within Quickwit's binary. + +That binary needs to be built and released by manually triggerring the `publish_lambda.yaml` github action. +By default this action only creates a draft release. You will need to manually publish the release for the url to be publicly downloadable. +The URL in quickwit-lambda-client/build.rs then needs to be manually updated. From 008a121d891197a89752b47b0ec7e2897ec19058 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Tue, 3 Feb 2026 13:21:31 +0100 Subject: [PATCH 16/22] adding logs --- .../quickwit-lambda-client/src/deployer.rs | 20 +++++++++++++------ quickwit/quickwit-serve/src/lib.rs | 3 ++- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/quickwit/quickwit-lambda-client/src/deployer.rs b/quickwit/quickwit-lambda-client/src/deployer.rs index 199ab7969a1..0796662278d 100644 --- a/quickwit/quickwit-lambda-client/src/deployer.rs +++ b/quickwit/quickwit-lambda-client/src/deployer.rs @@ -26,7 +26,7 @@ use aws_sdk_lambda::operation::get_function::GetFunctionOutput; use aws_sdk_lambda::primitives::Blob; use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; use quickwit_config::LambdaDeployConfig; -use tracing::{debug, info, warn}; +use tracing::{debug, error, info, warn}; use crate::error::{LambdaClientError, LambdaClientResult}; @@ -50,10 +50,10 @@ pub struct LambdaDeployer { impl LambdaDeployer { /// Create a new Lambda deployer using default AWS configuration. - pub async fn new() -> LambdaClientResult { + pub async fn new() -> Self { let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; let client = LambdaClient::new(&aws_config); - Ok(Self { client }) + Self { client } } /// Deploy or update the Lambda function. @@ -69,10 +69,12 @@ impl LambdaDeployer { match self.get_function(function_name).await { Ok(existing) => { + info!("update function if needed"); self.update_function_if_needed(function_name, &existing, deploy_config) .await } Err(LambdaClientError::NotFound(_)) => { + error!("function not found"); // Function doesn't exist, try to create it match self .create_function(function_name, role_arn, deploy_config) @@ -89,10 +91,16 @@ impl LambdaDeployer { self.update_function_if_needed(function_name, &existing, deploy_config) .await } - Err(e) => Err(e), + Err(e) => { + tracing::error!(e=?e, "lambda client error on creation"); + Err(e) + } } } - Err(e) => Err(e), + Err(e) => { + tracing::error!(e=?e, "lambda client error on get"); + Err(e) + } } } @@ -367,7 +375,7 @@ pub async fn deploy( function_name: &str, deploy_config: &LambdaDeployConfig, ) -> LambdaClientResult { - let lambda_deployer = LambdaDeployer::new().await?; + let lambda_deployer = LambdaDeployer::new().await; let lambda_arn = lambda_deployer.deploy(function_name, deploy_config).await?; info!("successfully deployed lambda function `{}`", lambda_arn); Ok(lambda_arn) diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index be0d7f64397..a0a523672aa 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1028,7 +1028,8 @@ async fn setup_searcher( use anyhow::Context; quickwit_lambda_client::deploy(&lambda_config.function_name, deploy_config) .await - .context("failed to deploy lambda function")?; + .context("failed to deploy lambda function") + .inspect_err(|err| error!(err=?err, "deploy lambda failed"))?; } use anyhow::Context; From 888e853bd8976e3fd21e00416c515d61938bb82f Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Tue, 3 Feb 2026 14:33:17 +0100 Subject: [PATCH 17/22] blop --- .../quickwit-lambda-client/src/deployer.rs | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/quickwit/quickwit-lambda-client/src/deployer.rs b/quickwit/quickwit-lambda-client/src/deployer.rs index 0796662278d..5d30b763ef5 100644 --- a/quickwit/quickwit-lambda-client/src/deployer.rs +++ b/quickwit/quickwit-lambda-client/src/deployer.rs @@ -22,7 +22,7 @@ use std::collections::HashMap; use aws_sdk_lambda::Client as LambdaClient; use aws_sdk_lambda::error::SdkError; use aws_sdk_lambda::operation::create_function::CreateFunctionError; -use aws_sdk_lambda::operation::get_function::GetFunctionOutput; +use aws_sdk_lambda::operation::get_function::{GetFunctionError, GetFunctionOutput}; use aws_sdk_lambda::primitives::Blob; use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; use quickwit_config::LambdaDeployConfig; @@ -332,21 +332,27 @@ impl LambdaDeployer { /// Get function details from AWS. async fn get_function(&self, name: &str) -> LambdaClientResult { - self.client + match self + .client .get_function() .function_name(name) .send() .await - .map_err(|e| { - // Check if it's a not found error - if e.to_string().contains("ResourceNotFoundException") - || e.to_string().contains("Function not found") - { - LambdaClientError::NotFound(name.to_string()) - } else { - LambdaClientError::Deployment(format!("failed to get function: {}", e)) - } - }) + { + Ok(output) => Ok(output), + Err(SdkError::ServiceError(err)) + if matches!(err.err(), GetFunctionError::ResourceNotFoundException(_)) => + { + Err(LambdaClientError::NotFound(name.to_string())) + } + Err(e) => { + error!(e=?e, "get function failed"); + Err(LambdaClientError::Deployment(format!( + "failed to get function: {}", + e + ))) + } + } } /// Build environment variables for the Lambda function. From 9284059414b55daec78eb3ef571ff00edef671e3 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Tue, 3 Feb 2026 15:56:47 +0100 Subject: [PATCH 18/22] lambda by default --- quickwit/quickwit-search/src/root.rs | 14 +++++++++++--- .../src/elasticsearch_api/rest_handler.rs | 2 +- .../quickwit-serve/src/search_api/rest_handler.rs | 2 +- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 2bb347942b2..4f1e37768af 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -46,7 +46,7 @@ use tantivy::aggregation::agg_result::AggregationResults; use tantivy::aggregation::intermediate_agg_result::IntermediateAggregationResults; use tantivy::collector::Collector; use tantivy::schema::{Field, FieldEntry, FieldType, Schema}; -use tracing::{debug, info, info_span, instrument}; +use tracing::{debug, error, info, info_span, instrument}; use crate::cluster_client::ClusterClient; use crate::collector::{QuickwitAggregations, make_merge_collector}; @@ -752,6 +752,10 @@ pub(crate) async fn search_partial_hits_phase( && searcher_context.searcher_config.lambda.is_some() { // Execute via Lambda + info!( + num_splits = split_metadatas.len(), + "Starting lambda execution for leaf search" + ); execute_leaf_search_via_lambda( search_request, indexes_metas_for_leaf_search, @@ -861,7 +865,11 @@ async fn execute_leaf_search_via_lambda( jobs_to_leaf_request(search_request, indexes_metas_for_leaf_search, batch_jobs); async move { let leaf_request = request?; - lambda_invoker.invoke_leaf_search(leaf_request).await + let response = lambda_invoker.invoke_leaf_search(leaf_request).await; + if let Err(err) = &response { + error!(error = %err, "Lambda invocation failed"); + } + response } }) .collect(); @@ -1301,7 +1309,7 @@ pub async fn root_search( if let Some(max_total_split_searches) = searcher_context.searcher_config.max_splits_per_search && max_total_split_searches < num_splits { - tracing::error!( + error!( num_splits, max_total_split_searches, index=?search_request.index_id_patterns, diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs index a605f3ea7f3..21488ddc08f 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs @@ -412,7 +412,7 @@ fn build_request_for_es_api( search_after, count_hits, ignore_missing_indexes, - execution_mode: 0, // Default to gRPC + execution_mode: 1, // Default to Lambda (RemoteFunction) }, has_doc_id_field, )) diff --git a/quickwit/quickwit-serve/src/search_api/rest_handler.rs b/quickwit/quickwit-serve/src/search_api/rest_handler.rs index e81f80d483b..f1ac6196998 100644 --- a/quickwit/quickwit-serve/src/search_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/search_api/rest_handler.rs @@ -265,7 +265,7 @@ pub fn search_request_from_api_request( search_after: None, count_hits: search_request.count_all.into(), ignore_missing_indexes: false, - execution_mode: 0, // Default to gRPC + execution_mode: 1, // Default to Lambda (RemoteFunction) }; Ok(search_request) } From 748f3264275b12738f6dc761d5e139f31a05197a Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Thu, 5 Feb 2026 12:37:58 +0100 Subject: [PATCH 19/22] blop --- quickwit/Cargo.lock | 2 + quickwit/quickwit-lambda-client/Cargo.toml | 2 + .../quickwit-lambda-client/src/deployer.rs | 139 +++++++++--------- quickwit/quickwit-lambda-client/src/error.rs | 98 +++++------- .../quickwit-lambda-client/src/invoker.rs | 10 +- quickwit/quickwit-lambda-client/src/lib.rs | 2 +- quickwit/quickwit-serve/src/lib.rs | 2 - 7 files changed, 119 insertions(+), 136 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 8134a01eace..b62f5341e1c 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7351,6 +7351,7 @@ dependencies = [ name = "quickwit-lambda-client" version = "0.8.0" dependencies = [ + "anyhow", "async-trait", "aws-config", "aws-sdk-lambda", @@ -7361,6 +7362,7 @@ dependencies = [ "quickwit-proto", "quickwit-search", "serde_json", + "thiserror 2.0.17", "tokio", "tracing", "ureq 3.1.4", diff --git a/quickwit/quickwit-lambda-client/Cargo.toml b/quickwit/quickwit-lambda-client/Cargo.toml index bdd0c1bb692..8d592fccda1 100644 --- a/quickwit/quickwit-lambda-client/Cargo.toml +++ b/quickwit/quickwit-lambda-client/Cargo.toml @@ -11,7 +11,9 @@ authors.workspace = true license.workspace = true [dependencies] +anyhow = { workspace = true } async-trait = { workspace = true } +thiserror = { workspace = true } aws-config = { workspace = true } aws-sdk-lambda = { workspace = true } base64 = { workspace = true } diff --git a/quickwit/quickwit-lambda-client/src/deployer.rs b/quickwit/quickwit-lambda-client/src/deployer.rs index 5d30b763ef5..ed7ebd85c1b 100644 --- a/quickwit/quickwit-lambda-client/src/deployer.rs +++ b/quickwit/quickwit-lambda-client/src/deployer.rs @@ -28,7 +28,7 @@ use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; use quickwit_config::LambdaDeployConfig; use tracing::{debug, error, info, warn}; -use crate::error::{LambdaClientError, LambdaClientResult}; +use crate::error::{LambdaDeployError, LambdaDeployResult}; /// Embedded Lambda binary (arm64, compressed). /// This is included at compile time. @@ -64,30 +64,45 @@ impl LambdaDeployer { &self, function_name: &str, deploy_config: &LambdaDeployConfig, - ) -> LambdaClientResult { + ) -> LambdaDeployResult { let role_arn = &deploy_config.execution_role_arn; - match self.get_function(function_name).await { - Ok(existing) => { + let function_info_opt = self.get_function(function_name).await.map_err(|err| { + tracing::error!(err=?err, "lambda client error on get"); + LambdaDeployError::Other(format!("failed to get function: {}", err)) + })?; + + match function_info_opt { + Some(existing) => { info!("update function if needed"); self.update_function_if_needed(function_name, &existing, deploy_config) .await } - Err(LambdaClientError::NotFound(_)) => { - error!("function not found"); + None => { + info!("function not found, creating"); // Function doesn't exist, try to create it match self .create_function(function_name, role_arn, deploy_config) .await { Ok(arn) => Ok(arn), - Err(LambdaClientError::ResourceConflict) => { + Err(LambdaDeployError::ResourceConflict) => { // Another node created the function concurrently, update instead - info!( + warn!( function_name = %function_name, "function was created concurrently by another node, updating instead" ); - let existing = self.get_function(function_name).await?; + let existing = self + .get_function(function_name) + .await + .map_err(|e| { + LambdaDeployError::Other(format!("failed to get function: {}", e)) + })? + .ok_or_else(|| { + LambdaDeployError::Other( + "function not found after concurrent creation".into(), + ) + })?; self.update_function_if_needed(function_name, &existing, deploy_config) .await } @@ -97,10 +112,6 @@ impl LambdaDeployer { } } } - Err(e) => { - tracing::error!(e=?e, "lambda client error on get"); - Err(e) - } } } @@ -114,7 +125,7 @@ impl LambdaDeployer { name: &str, role: &str, config: &LambdaDeployConfig, - ) -> LambdaClientResult { + ) -> LambdaDeployResult { info!( function_name = %name, role = %role, @@ -155,9 +166,7 @@ impl LambdaDeployer { Ok(output) => { let arn = output .function_arn() - .ok_or_else(|| { - LambdaClientError::Deployment("no function ARN returned".into()) - })? + .ok_or_else(|| LambdaDeployError::Other("no function ARN returned".into()))? .to_string(); info!(function_arn = %arn, "Lambda function created successfully"); Ok(arn) @@ -165,9 +174,9 @@ impl LambdaDeployer { Err(SdkError::ServiceError(err)) if matches!(err.err(), CreateFunctionError::ResourceConflictException(_)) => { - Err(LambdaClientError::ResourceConflict) + Err(LambdaDeployError::ResourceConflict) } - Err(e) => Err(LambdaClientError::Deployment(format!( + Err(e) => Err(LambdaDeployError::Other(format!( "failed to create function: {}", e ))), @@ -183,13 +192,11 @@ impl LambdaDeployer { name: &str, existing: &GetFunctionOutput, config: &LambdaDeployConfig, - ) -> LambdaClientResult { + ) -> LambdaDeployResult { let function_arn = existing .configuration() .and_then(|c| c.function_arn()) - .ok_or_else(|| { - LambdaClientError::Deployment("no function ARN in existing config".into()) - })? + .ok_or_else(|| LambdaDeployError::Other("no function ARN in existing config".into()))? .to_string(); if !self.needs_update(existing) { @@ -215,7 +222,7 @@ impl LambdaDeployer { .send() .await .map_err(|e| { - LambdaClientError::Deployment(format!("failed to update function code: {}", e)) + LambdaDeployError::Other(format!("failed to update function code: {}", e)) })?; // Wait for the update to complete before updating configuration @@ -236,10 +243,7 @@ impl LambdaDeployer { .send() .await .map_err(|e| { - LambdaClientError::Deployment(format!( - "failed to update function configuration: {}", - e - )) + LambdaDeployError::Other(format!("failed to update function configuration: {}", e)) })?; // Wait for config update to complete before updating tags @@ -253,7 +257,7 @@ impl LambdaDeployer { .send() .await .map_err(|e| { - LambdaClientError::Deployment(format!("failed to update function tags: {}", e)) + LambdaDeployError::Other(format!("failed to update function tags: {}", e)) })?; info!(function_arn = %function_arn, "Lambda function updated successfully"); @@ -261,10 +265,16 @@ impl LambdaDeployer { } /// Wait for function update to complete. - async fn wait_for_update_complete(&self, name: &str) -> LambdaClientResult<()> { + async fn wait_for_update_complete(&self, name: &str) -> LambdaDeployResult<()> { // Poll until the function state is Active and LastUpdateStatus is Successful for _ in 0..60 { - let output = self.get_function(name).await?; + let output = self + .get_function(name) + .await + .map_err(|e| LambdaDeployError::Other(format!("failed to get function: {}", e)))? + .ok_or_else(|| { + LambdaDeployError::Other("function not found while waiting for update".into()) + })?; if let Some(config) = output.configuration() { let state = config.state(); let last_update_status = config.last_update_status(); @@ -278,7 +288,7 @@ impl LambdaDeployer { let reason = config .last_update_status_reason() .unwrap_or("unknown reason"); - return Err(LambdaClientError::Deployment(format!( + return Err(LambdaDeployError::Other(format!( "function update failed: {}", reason ))); @@ -295,7 +305,7 @@ impl LambdaDeployer { } } } - Err(LambdaClientError::Deployment( + Err(LambdaDeployError::Other( "timeout waiting for function update to complete".into(), )) } @@ -304,53 +314,42 @@ impl LambdaDeployer { fn needs_update(&self, existing: &GetFunctionOutput) -> bool { let current_version = env!("CARGO_PKG_VERSION"); - let deployed_version = existing.tags().and_then(|t| t.get(VERSION_TAG_KEY)); + let Some(tags) = existing.tags() else { + warn!("no tags found on deployed function, update needed"); + return true; + }; - match deployed_version { - Some(version) if version == current_version => { - debug!( - deployed_version = %version, - current_version = %current_version, - "versions match, no update needed" - ); - false - } - Some(version) => { - info!( - deployed_version = %version, - current_version = %current_version, - "version mismatch, update needed" - ); - true - } - None => { - warn!("no version tag found on deployed function, update needed"); - true - } + let Some(deployed_version) = tags.get(VERSION_TAG_KEY) else { + warn!("no version tag found on deployed function, update needed"); + return true; + }; + + if deployed_version == current_version { + info!( + version = %deployed_version, + "versions match, no update needed" + ); + return false; } + + // TODO + true } /// Get function details from AWS. - async fn get_function(&self, name: &str) -> LambdaClientResult { - match self - .client - .get_function() - .function_name(name) - .send() - .await - { - Ok(output) => Ok(output), + /// + /// Returns `Ok(None)` if the function does not exist. + async fn get_function(&self, name: &str) -> anyhow::Result> { + match self.client.get_function().function_name(name).send().await { + Ok(output) => Ok(Some(output)), Err(SdkError::ServiceError(err)) if matches!(err.err(), GetFunctionError::ResourceNotFoundException(_)) => { - Err(LambdaClientError::NotFound(name.to_string())) + Ok(None) } Err(e) => { error!(e=?e, "get function failed"); - Err(LambdaClientError::Deployment(format!( - "failed to get function: {}", - e - ))) + anyhow::bail!("failed to get function: {}", e) } } } @@ -380,7 +379,7 @@ impl LambdaDeployer { pub async fn deploy( function_name: &str, deploy_config: &LambdaDeployConfig, -) -> LambdaClientResult { +) -> LambdaDeployResult { let lambda_deployer = LambdaDeployer::new().await; let lambda_arn = lambda_deployer.deploy(function_name, deploy_config).await?; info!("successfully deployed lambda function `{}`", lambda_arn); diff --git a/quickwit/quickwit-lambda-client/src/error.rs b/quickwit/quickwit-lambda-client/src/error.rs index eef384166e5..6b45ce969fe 100644 --- a/quickwit/quickwit-lambda-client/src/error.rs +++ b/quickwit/quickwit-lambda-client/src/error.rs @@ -12,79 +12,61 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::fmt; - use quickwit_search::SearchError; +use thiserror::Error; -/// Result type for Lambda client operations. -pub type LambdaClientResult = Result; +/// Result type for Lambda deployment operations. +pub type LambdaDeployResult = Result; -/// Errors that can occur during Lambda client operations. -#[derive(Debug)] -pub enum LambdaClientError { - /// Error during Lambda invocation. - Invocation(String), - /// Error serializing/deserializing protobuf. - Serialization(String), - /// Lambda function returned an error. - FunctionError(String), - /// Configuration error. - Configuration(String), - /// Internal error. - Internal(String), +/// Errors that can occur during Lambda function deployment. +#[derive(Debug, Error)] +pub enum LambdaDeployError { /// Resource conflict (e.g., function already exists during concurrent create). + #[error("resource conflict: Lambda function already exists")] ResourceConflict, - /// Error during Lambda function deployment. - Deployment(String), - /// Lambda function not found. - NotFound(String), -} -impl fmt::Display for LambdaClientError { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - match self { - LambdaClientError::Invocation(msg) => write!(f, "Lambda invocation error: {}", msg), - LambdaClientError::Serialization(msg) => write!(f, "Serialization error: {}", msg), - LambdaClientError::FunctionError(msg) => write!(f, "Lambda function error: {}", msg), - LambdaClientError::Configuration(msg) => write!(f, "Configuration error: {}", msg), - LambdaClientError::Internal(msg) => write!(f, "Internal error: {}", msg), - LambdaClientError::ResourceConflict => { - write!(f, "Resource conflict: function already exists") - } - LambdaClientError::Deployment(msg) => write!(f, "Deployment error: {}", msg), - LambdaClientError::NotFound(name) => write!(f, "Lambda function not found: {}", name), - } - } + /// General deployment error. + #[error("failed to deploy Lambda function: {0}")] + Other(String), } -impl std::error::Error for LambdaClientError {} +/// Result type for Lambda invoker operations. +pub type InvokerResult = Result; -impl From for LambdaClientError { - fn from(err: prost::DecodeError) -> Self { - LambdaClientError::Serialization(format!("Protobuf decode error: {}", err)) - } -} +/// Errors that can occur during Lambda invoker setup or invocation. +#[derive(Debug, Error)] +pub enum InvokerError { + /// Configuration or validation error. + #[error("Lambda configuration error: {0}")] + Configuration(String), -impl From for LambdaClientError { - fn from(err: prost::EncodeError) -> Self { - LambdaClientError::Serialization(format!("Protobuf encode error: {}", err)) - } -} + /// Error during Lambda invocation. + #[error("Lambda invocation failed: {0}")] + Invocation(String), -impl From for LambdaClientError { - fn from(err: base64::DecodeError) -> Self { - LambdaClientError::Serialization(format!("Base64 decode error: {}", err)) - } + /// Error serializing/deserializing data. + #[error("serialization error: {0}")] + Serialization(#[from] SerializationError), } -impl From for LambdaClientError { - fn from(err: serde_json::Error) -> Self { - LambdaClientError::Serialization(format!("JSON error: {}", err)) - } +/// Errors that can occur during serialization/deserialization. +#[derive(Debug, Error)] +pub enum SerializationError { + #[error("protobuf decode error: {0}")] + ProtobufDecode(#[from] prost::DecodeError), + + #[error("protobuf encode error: {0}")] + ProtobufEncode(#[from] prost::EncodeError), + + #[error("base64 decode error: {0}")] + Base64Decode(#[from] base64::DecodeError), + + #[error("JSON error: {0}")] + Json(#[from] serde_json::Error), } -impl From for SearchError { - fn from(err: LambdaClientError) -> Self { +impl From for SearchError { + fn from(err: InvokerError) -> Self { SearchError::Internal(err.to_string()) } } diff --git a/quickwit/quickwit-lambda-client/src/invoker.rs b/quickwit/quickwit-lambda-client/src/invoker.rs index 7837e2456d4..a8d0bcc3e18 100644 --- a/quickwit/quickwit-lambda-client/src/invoker.rs +++ b/quickwit/quickwit-lambda-client/src/invoker.rs @@ -26,14 +26,14 @@ use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; use quickwit_search::{RemoteFunctionInvoker, SearchError}; use tracing::{debug, instrument}; -use crate::error::{LambdaClientError, LambdaClientResult}; +use crate::error::{InvokerError, InvokerResult}; /// Create a Lambda invoker for remote leaf search execution. /// /// This creates and validates an AWS Lambda invoker that implements `RemoteFunctionInvoker`. pub async fn create_lambda_invoker( config: &LambdaConfig, -) -> LambdaClientResult> { +) -> InvokerResult> { let invoker = AwsLambdaInvoker::new(config).await?; invoker.validate().await?; Ok(Arc::new(invoker)) @@ -47,7 +47,7 @@ struct AwsLambdaInvoker { impl AwsLambdaInvoker { /// Create a new AWS Lambda invoker with the given configuration. - async fn new(config: &LambdaConfig) -> LambdaClientResult { + async fn new(config: &LambdaConfig) -> InvokerResult { let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; let client = LambdaClient::new(&aws_config); @@ -59,14 +59,14 @@ impl AwsLambdaInvoker { /// Validate that the Lambda function exists and is invocable. /// Uses DryRun invocation type - validates without executing. - async fn validate(&self) -> LambdaClientResult<()> { + async fn validate(&self) -> InvokerResult<()> { let request = self .client .invoke() .function_name(&self.function_name) .invocation_type(InvocationType::DryRun); request.send().await.map_err(|e| { - LambdaClientError::Configuration(format!( + InvokerError::Configuration(format!( "Failed to validate Lambda function '{}': {}", self.function_name, e )) diff --git a/quickwit/quickwit-lambda-client/src/lib.rs b/quickwit/quickwit-lambda-client/src/lib.rs index b404200ca25..99bab8f300f 100644 --- a/quickwit/quickwit-lambda-client/src/lib.rs +++ b/quickwit/quickwit-lambda-client/src/lib.rs @@ -23,7 +23,7 @@ mod error; mod invoker; pub use deployer::deploy; -pub use error::{LambdaClientError, LambdaClientResult}; +pub use error::{InvokerError, InvokerResult, LambdaDeployError, LambdaDeployResult}; pub use invoker::create_lambda_invoker; // Re-export payload types from server crate for convenience pub use quickwit_lambda_server::{LeafSearchPayload, LeafSearchResponsePayload}; diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index a0a523672aa..4f007ec255a 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1025,14 +1025,12 @@ async fn setup_searcher( // Auto-deploy Lambda function if configured if let Some(deploy_config) = &lambda_config.auto_deploy { info!("auto-deploying Lambda function"); - use anyhow::Context; quickwit_lambda_client::deploy(&lambda_config.function_name, deploy_config) .await .context("failed to deploy lambda function") .inspect_err(|err| error!(err=?err, "deploy lambda failed"))?; } - use anyhow::Context; let invoker = quickwit_lambda_client::create_lambda_invoker(lambda_config) .await .context("failed to initialize AWS Lambda invoker")?; From 4cd7ccab9fbe6a3d472e304601f5d35d6c61c47d Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Fri, 6 Feb 2026 13:47:06 +0100 Subject: [PATCH 20/22] deploy versioning, invoke and added metrics. --- quickwit/CLAUDE.md | 16 + quickwit/Cargo.lock | 4 + quickwit/quickwit-lambda-client/Cargo.toml | 4 + quickwit/quickwit-lambda-client/build.rs | 11 + quickwit/quickwit-lambda-client/src/deploy.rs | 600 ++++++++++++++++++ .../quickwit-lambda-client/src/deployer.rs | 421 ------------ .../quickwit-lambda-client/src/invoker.rs | 80 ++- quickwit/quickwit-lambda-client/src/lib.rs | 18 +- .../quickwit-lambda-client/src/metrics.rs | 82 +++ .../src/{remote_function.rs => invoker.rs} | 2 +- quickwit/quickwit-search/src/lib.rs | 6 +- quickwit/quickwit-search/src/root.rs | 12 +- quickwit/quickwit-search/src/service.rs | 6 +- quickwit/quickwit-serve/src/lib.rs | 22 +- 14 files changed, 812 insertions(+), 472 deletions(-) create mode 100644 quickwit/CLAUDE.md create mode 100644 quickwit/quickwit-lambda-client/src/deploy.rs delete mode 100644 quickwit/quickwit-lambda-client/src/deployer.rs create mode 100644 quickwit/quickwit-lambda-client/src/metrics.rs rename quickwit/quickwit-search/src/{remote_function.rs => invoker.rs} (95%) diff --git a/quickwit/CLAUDE.md b/quickwit/CLAUDE.md new file mode 100644 index 00000000000..69770162a01 --- /dev/null +++ b/quickwit/CLAUDE.md @@ -0,0 +1,16 @@ +# Quickwit Claude Guidelines + +## Code Formatting + +Run `make fmt` to check and fix code formatting. This command performs three checks: + +1. **Rust formatting**: Ensures Rust code is properly formatted (via `cargo fmt`) +2. **License headers**: Checks that files are prepended with the correct LICENSE header +3. **Log format policy**: Checks that log statements follow our format rules: + - No trailing punctuation in log messages + - No uppercase for the first character of log messages + - See `scripts/check_log_format.sh` for details + +### Quick Fix + +Use `/fmt` to automatically run format checks and see issues. diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index b62f5341e1c..d6b76dc61b6 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7356,12 +7356,16 @@ dependencies = [ "aws-config", "aws-sdk-lambda", "base64 0.22.1", + "hex", + "once_cell", "prost 0.14.1", + "quickwit-common", "quickwit-config", "quickwit-lambda-server", "quickwit-proto", "quickwit-search", "serde_json", + "sha1", "thiserror 2.0.17", "tokio", "tracing", diff --git a/quickwit/quickwit-lambda-client/Cargo.toml b/quickwit/quickwit-lambda-client/Cargo.toml index 8d592fccda1..b683006574c 100644 --- a/quickwit/quickwit-lambda-client/Cargo.toml +++ b/quickwit/quickwit-lambda-client/Cargo.toml @@ -19,15 +19,19 @@ aws-sdk-lambda = { workspace = true } base64 = { workspace = true } prost = { workspace = true } serde_json = { workspace = true } +once_cell = { workspace = true } tokio = { workspace = true } tracing = { workspace = true } +quickwit-common = { workspace = true } quickwit-config = { workspace = true } quickwit-lambda-server = { workspace = true } quickwit-proto = { workspace = true } quickwit-search = { workspace = true } [build-dependencies] +hex = { workspace = true } +sha1 = "0.10" ureq = { workspace = true } [features] diff --git a/quickwit/quickwit-lambda-client/build.rs b/quickwit/quickwit-lambda-client/build.rs index f18ca019aea..29f6d2407c3 100644 --- a/quickwit/quickwit-lambda-client/build.rs +++ b/quickwit/quickwit-lambda-client/build.rs @@ -24,6 +24,8 @@ use std::fs::File; use std::io::Write; use std::path::PathBuf; +use sha1::{Digest, Sha1}; + /// URL to download the pre-built Lambda zip from GitHub releases. /// This should be updated when a new Lambda binary is released. const LAMBDA_ZIP_URL: &str = @@ -54,6 +56,15 @@ fn main() { zip_path, data.len() ); + + // Compute SHA1 hash of the zip and export as environment variable. + // This is used to create a unique qualifier for Lambda versioning. + let mut hasher = Sha1::new(); + hasher.update(&data); + let sha1_hash = hasher.finalize(); + let sha1_short = hex::encode(&sha1_hash[..4]); // First 8 hex chars + println!("cargo:rustc-env=LAMBDA_BINARY_SHA1={}", sha1_short); + println!("cargo:warning=Lambda binary SHA1 (short): {}", sha1_short); } Err(e) => { panic!("Failed to download Lambda zip: {}", e); diff --git a/quickwit/quickwit-lambda-client/src/deploy.rs b/quickwit/quickwit-lambda-client/src/deploy.rs new file mode 100644 index 00000000000..ad9e2bd75c6 --- /dev/null +++ b/quickwit/quickwit-lambda-client/src/deploy.rs @@ -0,0 +1,600 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! Lambda function deployment for auto-deploy feature. +//! +//! This module provides functionality to automatically deploy or update +//! the Lambda function used for leaf search operations. +//! +//! # Versioning Strategy +//! +//! We use AWS Lambda published versions with description-based identification: +//! - Each published version has a description like `quickwit:0_8_0-fa752891` +//! - We list versions to find one matching our qualifier +//! - We invoke the specific version number (not $LATEST) +//! - Old versions are garbage collected (keep current + top 5 most recent) + +use std::collections::HashMap; +use std::sync::{Arc, OnceLock}; + +use anyhow::{Context, anyhow}; +use aws_sdk_lambda::Client as LambdaClient; +use aws_sdk_lambda::error::SdkError; +use aws_sdk_lambda::primitives::Blob; +use aws_sdk_lambda::types::{ + Architecture, Environment, FunctionCode, LastUpdateStatus, Runtime, State, +}; +use quickwit_config::{LambdaConfig, LambdaDeployConfig}; +use quickwit_search::LambdaLeafSearchInvoker; +use tracing::{debug, info, warn}; + +use crate::invoker::create_lambda_invoker_for_version; +use crate::metrics::LAMBDA_METRICS; + +/// Embedded Lambda binary (arm64, compressed). +/// This is included at compile time. +const LAMBDA_BINARY: &[u8] = include_bytes!(concat!(env!("OUT_DIR"), "/lambda_bootstrap.zip")); + +/// Prefix for version descriptions to identify Quickwit-managed versions. +const VERSION_DESCRIPTION_PREFIX: &str = "quickwit:"; + +/// Number of recent versions to keep during garbage collection (in addition to current). +const GC_KEEP_RECENT_VERSIONS: usize = 5; + +/// Returns the Lambda qualifier combining version and binary hash. +/// Format: "{quickwit_version}-{sha1_short}" with dots replaced by underscores. +/// Example: "0_8_0-fa752891" +fn lambda_qualifier() -> &'static str { + static LAMBDA_QUALIFIER: OnceLock = OnceLock::new(); + LAMBDA_QUALIFIER + .get_or_init(|| { + format!( + "{}-{}", + env!("CARGO_PKG_VERSION").replace('.', "_"), + env!("LAMBDA_BINARY_SHA1") + ) + }) + .as_str() +} + +/// Returns the version description for our qualifier. +fn version_description() -> String { + format!("{}{}", VERSION_DESCRIPTION_PREFIX, lambda_qualifier()) +} + +/// Get or deploy the Lambda function and return an invoker. +/// +/// This function: +/// 1. Lists existing Lambda versions to find one matching our description +/// 2. If not found, (and if a deploy config is provided) attempt to deploy the embedded Lambda +/// binary +/// 3. Garbage collects old versions (keeps current + 5 most recent) +/// 4. Returns an invoker configured to call the specific version +/// +/// The qualifier is computed from the Quickwit version and Lambda binary SHA1, +/// ensuring the deployed Lambda matches the embedded binary. +pub async fn try_get_or_deploy_invoker( + lambda_config: &LambdaConfig, +) -> anyhow::Result> { + let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; + let client = LambdaClient::new(&aws_config); + let function_name = &lambda_config.function_name; + let target_description = version_description(); + + info!( + function_name = %function_name, + qualifier = %lambda_qualifier(), + "Looking for Lambda function version" + ); + + let version = find_or_deploy_version( + &client, + function_name, + &target_description, + lambda_config.auto_deploy.as_ref(), + ) + .await?; + + // Step 3: Spawn background garbage collection (best effort, non-blocking) + let gc_client = client.clone(); + let gc_function_name = function_name.clone(); + let gc_version = version.clone(); + tokio::spawn(async move { + if let Err(e) = + garbage_collect_old_versions(&gc_client, &gc_function_name, &gc_version).await + { + warn!(error = %e, "Failed to garbage collect old Lambda versions"); + } + }); + + // Step 4: Create and return the invoker + let invoker = create_lambda_invoker_for_version(function_name, &version) + .await + .context("Failed to create Lambda invoker")?; + + info!("created the lambda invoker"); + + Ok(invoker) +} + +/// Find a Lambda version with a description matching our qualifier. +/// +/// If none is found and a deploy config is provided, attempt to deploy a new version. +/// +/// Returns the version number as a string (because it is a string on AWS side, e.g.: "7") if found. +async fn find_or_deploy_version( + client: &LambdaClient, + function_name: &str, + target_description: &str, + deploy_config: Option<&LambdaDeployConfig>, +) -> anyhow::Result { + if let Some(version) = find_matching_version(client, function_name, target_description).await? { + info!( + function_name = %function_name, + version = %version, + "Found existing Lambda version" + ); + return Ok(version); + } + + let deploy_config = deploy_config.with_context(|| { + format!( + "No Lambda version found with description '{}' and auto_deploy is not configured. \ + Either deploy the Lambda function manually or enable auto_deploy.", + target_description + ) + })?; + + info!( + function_name = %function_name, + "No matching version found, deploying Lambda function" + ); + + deploy_lambda_function(client, function_name, deploy_config).await +} + +async fn find_matching_version( + client: &LambdaClient, + function_name: &str, + target_description: &str, +) -> anyhow::Result> { + let mut marker: Option = None; + + loop { + let mut request = client + .list_versions_by_function() + .function_name(function_name); + + if let Some(m) = marker { + request = request.marker(m); + } + + let response = match request.send().await { + Ok(resp) => resp, + Err(SdkError::ServiceError(err)) if err.err().is_resource_not_found_exception() => { + debug!( + function_name = %function_name, + "Lambda function does not exist yet" + ); + return Ok(None); + } + Err(e) => { + return Err(anyhow!( + "Failed to list Lambda versions for '{}': {}", + function_name, + e + )); + } + }; + + for version in response.versions() { + if let Some(description) = version.description() { + if description == target_description { + if let Some(ver) = version.version() { + if ver != "$LATEST" { + return Ok(Some(ver.to_string())); + } + } + } + } + } + + marker = response.next_marker().map(|s| s.to_string()); + if marker.is_none() { + break; + } + } + + Ok(None) +} + +/// Deploy the Lambda function and publish a new version. +/// AWS's API is pretty terrible. +/// +/// Lambda's version are integer generated by AWS (we don't have control over them). +/// To publish a new version, we need to implement two paths: +/// - If the function doesn't exist yet, `create_function(publish=true)` atomically creates it and +/// publishes a version in one call. +/// - If the function already exists, we first update the code. We do not +/// publish because strangely the API call does not make it possible to change the +/// description. Updating the code has the effect of create a version $LATEST. +/// - We publish the version $LATEST. That's the moment AWS attributes a version number. +/// That call allows us to change the description. We pass the sha256 hash of the code +/// to ensure that $LATEST has not been overwritten by another concurrent update. +async fn deploy_lambda_function( + client: &LambdaClient, + function_name: &str, + deploy_config: &LambdaDeployConfig, +) -> anyhow::Result { + let result = deploy_lambda_function_inner(client, function_name, deploy_config).await; + + let status = if result.is_ok() { "success" } else { "error" }; + LAMBDA_METRICS + .deploy_total + .with_label_values([status]) + .inc(); + + result +} + +async fn deploy_lambda_function_inner( + client: &LambdaClient, + function_name: &str, + deploy_config: &LambdaDeployConfig, +) -> anyhow::Result { + // Fast path: create + publish atomically if the function doesn't exist yet. + if let Some(version) = try_create_function(client, function_name, deploy_config).await? { + return Ok(version); + } + + // Function already exists — update $LATEST, then publish with code_sha256 guard. + let code_sha256 = update_function_code(client, function_name).await?; + publish_version(client, function_name, &code_sha256).await +} + +/// Try to create the Lambda function with `publish=true`. +/// +/// Returns `Some(version)` if the function was created and published. +/// Returns `None` if the function already exists (`ResourceConflictException`). +async fn try_create_function( + client: &LambdaClient, + function_name: &str, + deploy_config: &LambdaDeployConfig, +) -> anyhow::Result> { + let memory_size_mb = (deploy_config.memory_size.as_u64() / (1024 * 1024)) as i32; + let timeout_secs = deploy_config.invocation_timeout_secs as i32; + let description = version_description(); + + info!( + function_name = %function_name, + memory_size_mb = memory_size_mb, + timeout_secs = timeout_secs, + "Attempting to create Lambda function" + ); + + let function_code = FunctionCode::builder() + .zip_file(Blob::new(LAMBDA_BINARY)) + .build(); + + let create_result = client + .create_function() + .function_name(function_name) + .runtime(Runtime::Providedal2023) + .role(&deploy_config.execution_role_arn) + .handler("bootstrap") + .description(&description) + .code(function_code) + .architectures(Architecture::Arm64) + .memory_size(memory_size_mb) + .timeout(timeout_secs) + .environment(build_environment()) + .set_tags(Some(build_tags())) + .publish(true) + .send() + .await; + + match create_result { + Ok(output) => { + let version = output + .version() + .ok_or_else(|| anyhow!("Created function has no version number"))? + .to_string(); + info!( + function_name = %function_name, + version = %version, + "Lambda function created and published" + ); + Ok(Some(version)) + } + Err(SdkError::ServiceError(err)) if err.err().is_resource_conflict_exception() => { + debug!( + function_name = %function_name, + "Lambda function already exists" + ); + Ok(None) + } + Err(e) => Err(anyhow!( + "Failed to create Lambda function '{}': {}", + function_name, + e + )), + } +} + +/// Update `$LATEST` to our embedded binary. +/// +/// Returns the `code_sha256` of the uploaded code, to be used as a guard +/// when publishing the version (detects if another process overwrote `$LATEST` +/// between our update and publish). +async fn update_function_code( + client: &LambdaClient, + function_name: &str, +) -> anyhow::Result { + info!( + function_name = %function_name, + "Updating Lambda function code to current binary" + ); + + let response = client + .update_function_code() + .function_name(function_name) + .zip_file(Blob::new(LAMBDA_BINARY)) + .architectures(Architecture::Arm64) + .send() + .await + .context("Failed to update Lambda function code")?; + + let code_sha256 = response + .code_sha256() + .ok_or_else(|| anyhow!("update_function_code response missing code_sha256"))? + .to_string(); + + wait_for_function_ready(client, function_name).await?; + + Ok(code_sha256) +} + +/// Publish a new immutable version from `$LATEST` with our description. +/// +/// The `code_sha256` parameter guards against races: if another process +/// overwrote `$LATEST` since our `update_function_code` call, AWS will +/// reject the publish. +/// +/// Returns the version number (e.g., "8"). +async fn publish_version( + client: &LambdaClient, + function_name: &str, + code_sha256: &str, +) -> anyhow::Result { + let description = version_description(); + + info!( + function_name = %function_name, + description = %description, + "Publishing new Lambda version" + ); + + let publish_response = client + .publish_version() + .function_name(function_name) + .description(&description) + .code_sha256(code_sha256) + .send() + .await + .context( + "Failed to publish Lambda version (code_sha256 mismatch means a concurrent deploy \ + race)", + )?; + + let version = publish_response + .version() + .context("Published version has no version number")? + .to_string(); + + info!( + function_name = %function_name, + version = %version, + "Lambda version published successfully" + ); + + Ok(version) +} + +/// Wait for the Lambda function to be ready. +/// +/// "Ready" means `State == Active` and no update is in progress +/// (`LastUpdateStatus` is absent or `Successful`). +/// +/// This matters because: +/// - After `create_function`: `State` transitions `Pending → Active` +/// - After `update_function_code`: `State` stays `Active` but `LastUpdateStatus` transitions +/// `InProgress → Successful` +async fn wait_for_function_ready(client: &LambdaClient, function_name: &str) -> anyhow::Result<()> { + const MAX_WAIT_ATTEMPTS: u32 = 30; + const WAIT_INTERVAL: tokio::time::Duration = tokio::time::Duration::from_secs(1); + + let mut interval = tokio::time::interval(WAIT_INTERVAL); + + for attempt in 0..MAX_WAIT_ATTEMPTS { + interval.tick().await; + + let response = client + .get_function() + .function_name(function_name) + .send() + .await + .context("Failed to get function status")?; + + let Some(config) = response.configuration() else { + continue; + }; + + // Check for terminal failure states. + if config.state() == Some(&State::Failed) { + let reason = config.state_reason().unwrap_or("Unknown reason"); + anyhow::bail!( + "Lambda function '{}' is in Failed state: {}", + function_name, + reason + ); + } + + let last_update_status: &LastUpdateStatus = config + .last_update_status() + .unwrap_or(&LastUpdateStatus::Successful); + + if last_update_status == &LastUpdateStatus::Failed { + let reason = config + .last_update_status_reason() + .unwrap_or("Unknown reason"); + anyhow::bail!( + "Lambda function '{}' last update failed: {}", + function_name, + reason + ); + } + + // Ready = Active state with no update in progress. + let is_active = config.state() == Some(&State::Active); + if is_active && last_update_status == &LastUpdateStatus::Successful { + info!( + function_name = %function_name, + attempts = attempt + 1, + "Lambda function is ready" + ); + return Ok(()); + } + + info!( + function_name = %function_name, + state = ?config.state(), + last_update_status = ?config.last_update_status(), + attempt = attempt + 1, + "Waiting for Lambda function to be ready" + ); + } + + anyhow::bail!( + "Lambda function '{}' did not become ready within {} seconds", + function_name, + MAX_WAIT_ATTEMPTS as u64 * WAIT_INTERVAL.as_secs() + ) +} + +/// Garbage collect old Lambda versions, keeping the current + 5 most recent. +async fn garbage_collect_old_versions( + client: &LambdaClient, + function_name: &str, + current_version: &str, +) -> anyhow::Result<()> { + let mut quickwit_lambda_versions: Vec<(u64, String)> = Vec::new(); + let mut marker: Option = None; + + // Collect all Quickwit-managed versions + loop { + let mut request = client + .list_versions_by_function() + .function_name(function_name); + + if let Some(m) = marker { + request = request.marker(m); + } + + let response = request + .send() + .await + .context("Failed to list Lambda versions for garbage collection")?; + + for version in response.versions() { + let Some(version_str) = version.version() else { + continue; + }; + if version_str == "$LATEST" { + continue; + } + // Only consider Quickwit-managed versions + let Some(description) = version.description() else { + continue; + }; + if description.starts_with(VERSION_DESCRIPTION_PREFIX) { + if let Ok(version_num) = version_str.parse::() { + quickwit_lambda_versions.push((version_num, version_str.to_string())); + } + } + } + + marker = response.next_marker().map(|s| s.to_string()); + if marker.is_none() { + break; + } + } + + // Sort by version number descending (most recent first) + quickwit_lambda_versions.sort(); + + // We keep the last 5 versions. + quickwit_lambda_versions.truncate( + quickwit_lambda_versions + .len() + .saturating_sub(GC_KEEP_RECENT_VERSIONS), + ); + + if let Some(pos) = quickwit_lambda_versions + .iter() + .position(|(_version, version_str)| version_str == current_version) + { + quickwit_lambda_versions.swap_remove(pos); + } + + // Delete old versions + for (version, version_str) in quickwit_lambda_versions { + info!( + function_name = %function_name, + version = %version_str, + "Deleting old Lambda version" + ); + + if let Err(e) = client + .delete_function() + .function_name(function_name) + .qualifier(&version_str) + .send() + .await + { + warn!( + function_name = %function_name, + version = %version, + error = %e, + "Failed to delete old Lambda version" + ); + } + } + + Ok(()) +} + +/// Build environment variables for the Lambda function. +fn build_environment() -> Environment { + let mut env_vars = HashMap::new(); + env_vars.insert("RUST_LOG".to_string(), "info".to_string()); + env_vars.insert("RUST_BACKTRACE".to_string(), "1".to_string()); + Environment::builder().set_variables(Some(env_vars)).build() +} + +/// Build tags for the Lambda function. +fn build_tags() -> HashMap { + let mut tags = HashMap::new(); + tags.insert("managed_by".to_string(), "quickwit".to_string()); + tags +} diff --git a/quickwit/quickwit-lambda-client/src/deployer.rs b/quickwit/quickwit-lambda-client/src/deployer.rs deleted file mode 100644 index ed7ebd85c1b..00000000000 --- a/quickwit/quickwit-lambda-client/src/deployer.rs +++ /dev/null @@ -1,421 +0,0 @@ -// Copyright 2021-Present Datadog, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -//! Lambda function deployment for auto-deploy feature. -//! -//! This module provides functionality to automatically deploy or update -//! the Lambda function used for leaf search operations. - -use std::collections::HashMap; - -use aws_sdk_lambda::Client as LambdaClient; -use aws_sdk_lambda::error::SdkError; -use aws_sdk_lambda::operation::create_function::CreateFunctionError; -use aws_sdk_lambda::operation::get_function::{GetFunctionError, GetFunctionOutput}; -use aws_sdk_lambda::primitives::Blob; -use aws_sdk_lambda::types::{Architecture, Environment, FunctionCode, Runtime}; -use quickwit_config::LambdaDeployConfig; -use tracing::{debug, error, info, warn}; - -use crate::error::{LambdaDeployError, LambdaDeployResult}; - -/// Embedded Lambda binary (arm64, compressed). -/// This is included at compile time. -const LAMBDA_BINARY: &[u8] = include_bytes!(concat!(env!("OUT_DIR"), "/lambda_bootstrap.zip")); - -/// Version tag key used to track deployed Quickwit version. -const VERSION_TAG_KEY: &str = "quickwit_version"; - -/// Description prefix for auto-deployed Lambda functions. -const FUNCTION_DESCRIPTION: &str = "Quickwit Lambda leaf search handler"; - -/// Lambda function deployer. -/// -/// Handles creating and updating Lambda functions for the auto-deploy feature. -/// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. -pub struct LambdaDeployer { - client: LambdaClient, -} - -impl LambdaDeployer { - /// Create a new Lambda deployer using default AWS configuration. - pub async fn new() -> Self { - let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; - let client = LambdaClient::new(&aws_config); - Self { client } - } - - /// Deploy or update the Lambda function. - /// - /// Safe for concurrent calls from multiple Quickwit nodes - CreateFunction is idempotent. - /// Returns the function ARN. - pub async fn deploy( - &self, - function_name: &str, - deploy_config: &LambdaDeployConfig, - ) -> LambdaDeployResult { - let role_arn = &deploy_config.execution_role_arn; - - let function_info_opt = self.get_function(function_name).await.map_err(|err| { - tracing::error!(err=?err, "lambda client error on get"); - LambdaDeployError::Other(format!("failed to get function: {}", err)) - })?; - - match function_info_opt { - Some(existing) => { - info!("update function if needed"); - self.update_function_if_needed(function_name, &existing, deploy_config) - .await - } - None => { - info!("function not found, creating"); - // Function doesn't exist, try to create it - match self - .create_function(function_name, role_arn, deploy_config) - .await - { - Ok(arn) => Ok(arn), - Err(LambdaDeployError::ResourceConflict) => { - // Another node created the function concurrently, update instead - warn!( - function_name = %function_name, - "function was created concurrently by another node, updating instead" - ); - let existing = self - .get_function(function_name) - .await - .map_err(|e| { - LambdaDeployError::Other(format!("failed to get function: {}", e)) - })? - .ok_or_else(|| { - LambdaDeployError::Other( - "function not found after concurrent creation".into(), - ) - })?; - self.update_function_if_needed(function_name, &existing, deploy_config) - .await - } - Err(e) => { - tracing::error!(e=?e, "lambda client error on creation"); - Err(e) - } - } - } - } - } - - /// Create the Lambda function. - /// - /// Note: CreateFunction is idempotent - if the function already exists, AWS returns - /// ResourceConflictException. Multiple Quickwit nodes starting simultaneously is safe; - /// one will succeed and others will fall back to update_function. - async fn create_function( - &self, - name: &str, - role: &str, - config: &LambdaDeployConfig, - ) -> LambdaDeployResult { - info!( - function_name = %name, - role = %role, - memory_size = %config.memory_size, - timeout_secs = config.invocation_timeout_secs, - "creating Lambda function" - ); - - let environment = self.build_environment(); - let tags = self.build_tags(); - - let result = self - .client - .create_function() - .function_name(name) - .runtime(Runtime::Providedal2023) - .architectures(Architecture::Arm64) - .handler("bootstrap") - .role(role) - .code( - FunctionCode::builder() - .zip_file(Blob::new(LAMBDA_BINARY)) - .build(), - ) - .memory_size((config.memory_size.as_u64() / (1024 * 1024)) as i32) - .timeout(config.invocation_timeout_secs as i32) - .environment(environment) - .description(format!( - "{} (v{})", - FUNCTION_DESCRIPTION, - env!("CARGO_PKG_VERSION") - )) - .set_tags(Some(tags)) - .send() - .await; - - match result { - Ok(output) => { - let arn = output - .function_arn() - .ok_or_else(|| LambdaDeployError::Other("no function ARN returned".into()))? - .to_string(); - info!(function_arn = %arn, "Lambda function created successfully"); - Ok(arn) - } - Err(SdkError::ServiceError(err)) - if matches!(err.err(), CreateFunctionError::ResourceConflictException(_)) => - { - Err(LambdaDeployError::ResourceConflict) - } - Err(e) => Err(LambdaDeployError::Other(format!( - "failed to create function: {}", - e - ))), - } - } - - /// Update the Lambda function if needed. - /// - /// Compares the deployed version tag with the current Quickwit version - /// and updates if they differ. - async fn update_function_if_needed( - &self, - name: &str, - existing: &GetFunctionOutput, - config: &LambdaDeployConfig, - ) -> LambdaDeployResult { - let function_arn = existing - .configuration() - .and_then(|c| c.function_arn()) - .ok_or_else(|| LambdaDeployError::Other("no function ARN in existing config".into()))? - .to_string(); - - if !self.needs_update(existing) { - debug!( - function_name = %name, - "Lambda function is up to date, skipping update" - ); - return Ok(function_arn); - } - - info!( - function_name = %name, - "updating Lambda function to version {}", - env!("CARGO_PKG_VERSION") - ); - - // Update function code - self.client - .update_function_code() - .function_name(name) - .zip_file(Blob::new(LAMBDA_BINARY)) - .architectures(Architecture::Arm64) - .send() - .await - .map_err(|e| { - LambdaDeployError::Other(format!("failed to update function code: {}", e)) - })?; - - // Wait for the update to complete before updating configuration - self.wait_for_update_complete(name).await?; - - // Update function configuration - self.client - .update_function_configuration() - .function_name(name) - .memory_size((config.memory_size.as_u64() / (1024 * 1024)) as i32) - .timeout(config.invocation_timeout_secs as i32) - .environment(self.build_environment()) - .description(format!( - "{} (v{})", - FUNCTION_DESCRIPTION, - env!("CARGO_PKG_VERSION") - )) - .send() - .await - .map_err(|e| { - LambdaDeployError::Other(format!("failed to update function configuration: {}", e)) - })?; - - // Wait for config update to complete before updating tags - self.wait_for_update_complete(name).await?; - - // Update tags - self.client - .tag_resource() - .resource(function_arn.clone()) - .set_tags(Some(self.build_tags())) - .send() - .await - .map_err(|e| { - LambdaDeployError::Other(format!("failed to update function tags: {}", e)) - })?; - - info!(function_arn = %function_arn, "Lambda function updated successfully"); - Ok(function_arn) - } - - /// Wait for function update to complete. - async fn wait_for_update_complete(&self, name: &str) -> LambdaDeployResult<()> { - // Poll until the function state is Active and LastUpdateStatus is Successful - for _ in 0..60 { - let output = self - .get_function(name) - .await - .map_err(|e| LambdaDeployError::Other(format!("failed to get function: {}", e)))? - .ok_or_else(|| { - LambdaDeployError::Other("function not found while waiting for update".into()) - })?; - if let Some(config) = output.configuration() { - let state = config.state(); - let last_update_status = config.last_update_status(); - - use aws_sdk_lambda::types::{LastUpdateStatus, State}; - match (state, last_update_status) { - (Some(State::Active), Some(LastUpdateStatus::Successful)) => { - return Ok(()); - } - (Some(State::Failed), _) | (_, Some(LastUpdateStatus::Failed)) => { - let reason = config - .last_update_status_reason() - .unwrap_or("unknown reason"); - return Err(LambdaDeployError::Other(format!( - "function update failed: {}", - reason - ))); - } - _ => { - debug!( - function_name = %name, - state = ?state, - last_update_status = ?last_update_status, - "waiting for function update to complete" - ); - tokio::time::sleep(tokio::time::Duration::from_secs(1)).await; - } - } - } - } - Err(LambdaDeployError::Other( - "timeout waiting for function update to complete".into(), - )) - } - - /// Check if the function needs to be updated based on version tag. - fn needs_update(&self, existing: &GetFunctionOutput) -> bool { - let current_version = env!("CARGO_PKG_VERSION"); - - let Some(tags) = existing.tags() else { - warn!("no tags found on deployed function, update needed"); - return true; - }; - - let Some(deployed_version) = tags.get(VERSION_TAG_KEY) else { - warn!("no version tag found on deployed function, update needed"); - return true; - }; - - if deployed_version == current_version { - info!( - version = %deployed_version, - "versions match, no update needed" - ); - return false; - } - - // TODO - true - } - - /// Get function details from AWS. - /// - /// Returns `Ok(None)` if the function does not exist. - async fn get_function(&self, name: &str) -> anyhow::Result> { - match self.client.get_function().function_name(name).send().await { - Ok(output) => Ok(Some(output)), - Err(SdkError::ServiceError(err)) - if matches!(err.err(), GetFunctionError::ResourceNotFoundException(_)) => - { - Ok(None) - } - Err(e) => { - error!(e=?e, "get function failed"); - anyhow::bail!("failed to get function: {}", e) - } - } - } - - /// Build environment variables for the Lambda function. - fn build_environment(&self) -> Environment { - let mut env_vars = HashMap::new(); - // Set reasonable defaults for logging - env_vars.insert("RUST_LOG".to_string(), "info".to_string()); - env_vars.insert("RUST_BACKTRACE".to_string(), "1".to_string()); - - Environment::builder().set_variables(Some(env_vars)).build() - } - - /// Build tags for the Lambda function. - fn build_tags(&self) -> HashMap { - let mut tags = HashMap::new(); - tags.insert( - VERSION_TAG_KEY.to_string(), - env!("CARGO_PKG_VERSION").to_string(), - ); - tags.insert("managed_by".to_string(), "quickwit".to_string()); - tags - } -} - -pub async fn deploy( - function_name: &str, - deploy_config: &LambdaDeployConfig, -) -> LambdaDeployResult { - let lambda_deployer = LambdaDeployer::new().await; - let lambda_arn = lambda_deployer.deploy(function_name, deploy_config).await?; - info!("successfully deployed lambda function `{}`", lambda_arn); - Ok(lambda_arn) -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_build_tags() { - let deployer = LambdaDeployer { - client: LambdaClient::from_conf( - aws_sdk_lambda::Config::builder() - .behavior_version(aws_sdk_lambda::config::BehaviorVersion::latest()) - .build(), - ), - }; - let tags = deployer.build_tags(); - assert!(tags.contains_key(VERSION_TAG_KEY)); - assert_eq!(tags.get("managed_by"), Some(&"quickwit".to_string())); - } - - #[test] - fn test_build_environment() { - let deployer = LambdaDeployer { - client: LambdaClient::from_conf( - aws_sdk_lambda::Config::builder() - .behavior_version(aws_sdk_lambda::config::BehaviorVersion::latest()) - .build(), - ), - }; - let env = deployer.build_environment(); - let vars = env.variables().unwrap(); - assert_eq!(vars.get("RUST_LOG"), Some(&"info".to_string())); - assert_eq!(vars.get("RUST_BACKTRACE"), Some(&"1".to_string())); - } -} diff --git a/quickwit/quickwit-lambda-client/src/invoker.rs b/quickwit/quickwit-lambda-client/src/invoker.rs index a8d0bcc3e18..f078784ec5f 100644 --- a/quickwit/quickwit-lambda-client/src/invoker.rs +++ b/quickwit/quickwit-lambda-client/src/invoker.rs @@ -20,21 +20,23 @@ use aws_sdk_lambda::primitives::Blob; use aws_sdk_lambda::types::InvocationType; use base64::prelude::*; use prost::Message; -use quickwit_config::LambdaConfig; use quickwit_lambda_server::{LeafSearchPayload, LeafSearchResponsePayload}; use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; -use quickwit_search::{RemoteFunctionInvoker, SearchError}; -use tracing::{debug, instrument}; +use quickwit_search::{LambdaLeafSearchInvoker, SearchError}; +use tracing::{debug, info, instrument}; use crate::error::{InvokerError, InvokerResult}; +use crate::metrics::LAMBDA_METRICS; -/// Create a Lambda invoker for remote leaf search execution. +/// Create a Lambda invoker for a specific version. /// -/// This creates and validates an AWS Lambda invoker that implements `RemoteFunctionInvoker`. -pub async fn create_lambda_invoker( - config: &LambdaConfig, -) -> InvokerResult> { - let invoker = AwsLambdaInvoker::new(config).await?; +/// The version number is used as the qualifier when invoking, ensuring we call +/// the exact published version (not $LATEST). +pub(crate) async fn create_lambda_invoker_for_version( + function_name: &str, + version: &str, +) -> InvokerResult> { + let invoker = AwsLambdaInvoker::new(function_name, version).await?; invoker.validate().await?; Ok(Arc::new(invoker)) } @@ -43,45 +45,76 @@ pub async fn create_lambda_invoker( struct AwsLambdaInvoker { client: LambdaClient, function_name: String, + /// The version number to invoke (e.g., "7", "12"). + version: String, } impl AwsLambdaInvoker { - /// Create a new AWS Lambda invoker with the given configuration. - async fn new(config: &LambdaConfig) -> InvokerResult { + /// Create a new AWS Lambda invoker for a specific version. + async fn new(function_name: &str, version: &str) -> InvokerResult { let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; let client = LambdaClient::new(&aws_config); Ok(Self { client, - function_name: config.function_name.clone(), + function_name: function_name.to_string(), + version: version.to_string(), }) } - /// Validate that the Lambda function exists and is invocable. + /// Validate that the Lambda function version exists and is invocable. /// Uses DryRun invocation type - validates without executing. async fn validate(&self) -> InvokerResult<()> { + info!("lambda invoker dry run"); let request = self .client .invoke() .function_name(&self.function_name) + .qualifier(&self.version) .invocation_type(InvocationType::DryRun); + request.send().await.map_err(|e| { InvokerError::Configuration(format!( - "Failed to validate Lambda function '{}': {}", - self.function_name, e + "Failed to validate Lambda function '{}:{}': {}", + self.function_name, self.version, e )) })?; + info!("the lambda invoker dry run was successful"); Ok(()) } } #[async_trait] -impl RemoteFunctionInvoker for AwsLambdaInvoker { - #[instrument(skip(self, request), fields(function_name = %self.function_name))] +impl LambdaLeafSearchInvoker for AwsLambdaInvoker { + #[instrument(skip(self, request), fields(function_name = %self.function_name, version = %self.version))] async fn invoke_leaf_search( &self, request: LeafSearchRequest, + ) -> Result { + let start = std::time::Instant::now(); + + let result = self.invoke_leaf_search_inner(request).await; + + let elapsed = start.elapsed().as_secs_f64(); + let status = if result.is_ok() { "success" } else { "error" }; + LAMBDA_METRICS + .leaf_search_requests_total + .with_label_values([status]) + .inc(); + LAMBDA_METRICS + .leaf_search_duration_seconds + .with_label_values([status]) + .observe(elapsed); + + result + } +} + +impl AwsLambdaInvoker { + async fn invoke_leaf_search_inner( + &self, + request: LeafSearchRequest, ) -> Result { // Serialize request to protobuf bytes, then base64 encode let request_bytes = request.encode_to_vec(); @@ -92,18 +125,25 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { let payload_json = serde_json::to_vec(&payload) .map_err(|e| SearchError::Internal(format!("JSON serialization error: {}", e)))?; + LAMBDA_METRICS + .leaf_search_request_payload_size_bytes + .observe(payload_json.len() as f64); + debug!( payload_size = payload_json.len(), + version = %self.version, "Invoking Lambda function" ); - // Invoke Lambda synchronously (RequestResponse) + // Invoke the specific version let invoke_builder = self .client .invoke() .function_name(&self.function_name) + .qualifier(&self.version) .invocation_type(InvocationType::RequestResponse) .payload(Blob::new(payload_json)); + let response = invoke_builder .send() .await @@ -126,6 +166,10 @@ impl RemoteFunctionInvoker for AwsLambdaInvoker { .payload() .ok_or_else(|| SearchError::Internal("No response payload from Lambda".into()))?; + LAMBDA_METRICS + .leaf_search_response_payload_size_bytes + .observe(response_payload.as_ref().len() as f64); + let lambda_response: LeafSearchResponsePayload = serde_json::from_slice(response_payload.as_ref()) .map_err(|e| SearchError::Internal(format!("JSON deserialization error: {}", e)))?; diff --git a/quickwit/quickwit-lambda-client/src/lib.rs b/quickwit/quickwit-lambda-client/src/lib.rs index 99bab8f300f..ead3aef464b 100644 --- a/quickwit/quickwit-lambda-client/src/lib.rs +++ b/quickwit/quickwit-lambda-client/src/lib.rs @@ -17,13 +17,25 @@ //! This crate provides: //! - An AWS Lambda implementation of the `RemoteFunctionInvoker` trait //! - Auto-deployment functionality for Lambda functions +//! +//! # Usage +//! +//! Use `get_or_deploy_invoker` to get an invoker that will automatically deploy +//! the Lambda function if needed: +//! +//! ```ignore +//! let invoker = get_or_deploy_invoker(&function_name, &deploy_config).await?; +//! ``` -mod deployer; +// mod deploy; mod error; mod invoker; +mod metrics; -pub use deployer::deploy; pub use error::{InvokerError, InvokerResult, LambdaDeployError, LambdaDeployResult}; -pub use invoker::create_lambda_invoker; +pub use metrics::LAMBDA_METRICS; // Re-export payload types from server crate for convenience pub use quickwit_lambda_server::{LeafSearchPayload, LeafSearchResponsePayload}; +mod deploy; + +pub use deploy::try_get_or_deploy_invoker; diff --git a/quickwit/quickwit-lambda-client/src/metrics.rs b/quickwit/quickwit-lambda-client/src/metrics.rs new file mode 100644 index 00000000000..89eb92f5a07 --- /dev/null +++ b/quickwit/quickwit-lambda-client/src/metrics.rs @@ -0,0 +1,82 @@ +// Copyright 2021-Present Datadog, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// See https://prometheus.io/docs/practices/naming/ + +use once_cell::sync::Lazy; +use quickwit_common::metrics::{ + Histogram, HistogramVec, IntCounterVec, exponential_buckets, new_counter_vec, new_histogram, + new_histogram_vec, +}; + +/// From 0.008s to 131s +fn duration_buckets() -> Vec { + exponential_buckets(0.008, 2.0, 15).unwrap() +} + +/// From 1KB to 50MB +fn payload_size_buckets() -> Vec { + exponential_buckets(1024.0, 4.0, 8).unwrap() +} + +pub struct LambdaMetrics { + pub leaf_search_requests_total: IntCounterVec<1>, + pub leaf_search_duration_seconds: HistogramVec<1>, + pub leaf_search_request_payload_size_bytes: Histogram, + pub leaf_search_response_payload_size_bytes: Histogram, + pub deploy_total: IntCounterVec<1>, +} + +impl Default for LambdaMetrics { + fn default() -> Self { + LambdaMetrics { + leaf_search_requests_total: new_counter_vec( + "leaf_search_requests_total", + "Total number of Lambda leaf search invocations.", + "lambda", + &[], + ["status"], + ), + leaf_search_duration_seconds: new_histogram_vec( + "leaf_search_duration_seconds", + "Duration of Lambda leaf search invocations in seconds.", + "lambda", + &[], + ["status"], + duration_buckets(), + ), + leaf_search_request_payload_size_bytes: new_histogram( + "leaf_search_request_payload_size_bytes", + "Size of the request payload sent to Lambda in bytes.", + "lambda", + payload_size_buckets(), + ), + leaf_search_response_payload_size_bytes: new_histogram( + "leaf_search_response_payload_size_bytes", + "Size of the response payload received from Lambda in bytes.", + "lambda", + payload_size_buckets(), + ), + deploy_total: new_counter_vec( + "deploy_total", + "Total number of Lambda deployment attempts.", + "lambda", + &[], + ["status"], + ), + } + } +} + +pub static LAMBDA_METRICS: Lazy = Lazy::new(LambdaMetrics::default); diff --git a/quickwit/quickwit-search/src/remote_function.rs b/quickwit/quickwit-search/src/invoker.rs similarity index 95% rename from quickwit/quickwit-search/src/remote_function.rs rename to quickwit/quickwit-search/src/invoker.rs index 75da7a29207..7ee9ceb1c09 100644 --- a/quickwit/quickwit-search/src/remote_function.rs +++ b/quickwit/quickwit-search/src/invoker.rs @@ -24,7 +24,7 @@ use crate::SearchError; /// This abstraction allows different cloud providers to be supported. /// Implementations are provided by the `quickwit-lambda` crate. #[async_trait] -pub trait RemoteFunctionInvoker: Send + Sync + 'static { +pub trait LambdaLeafSearchInvoker: Send + Sync + 'static { /// Invoke the remote function with a LeafSearchRequest. async fn invoke_leaf_search( &self, diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 34449309e24..96721d83adb 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -24,6 +24,7 @@ mod error; mod fetch_docs; mod find_trace_ids_collector; +mod invoker; /// Leaf search operations. pub mod leaf; mod leaf_cache; @@ -31,7 +32,6 @@ mod list_fields; mod list_fields_cache; mod list_terms; mod metrics_trackers; -mod remote_function; mod retry; mod root; mod scroll_context; @@ -83,7 +83,7 @@ pub use crate::client::{ pub use crate::cluster_client::ClusterClient; pub use crate::error::{SearchError, parse_grpc_error}; use crate::fetch_docs::fetch_docs; -pub use crate::remote_function::RemoteFunctionInvoker; +pub use crate::invoker::LambdaLeafSearchInvoker; pub use crate::root::{ IndexMetasForLeafSearch, SearchJob, check_all_index_metadata_found, jobs_to_leaf_request, root_search, search_plan, @@ -264,7 +264,7 @@ pub async fn start_searcher_service( storage_resolver: StorageResolver, search_job_placer: SearchJobPlacer, searcher_context: Arc, - lambda_invoker: Option>, + lambda_invoker: Option>, ) -> anyhow::Result> { let cluster_client = ClusterClient::new(search_job_placer); let search_service = Arc::new(SearchServiceImpl::new( diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 4f1e37768af..150c86418eb 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -50,8 +50,8 @@ use tracing::{debug, error, info, info_span, instrument}; use crate::cluster_client::ClusterClient; use crate::collector::{QuickwitAggregations, make_merge_collector}; +use crate::invoker::LambdaLeafSearchInvoker; use crate::metrics_trackers::{RootSearchMetricsFuture, RootSearchMetricsStep}; -use crate::remote_function::RemoteFunctionInvoker; use crate::scroll_context::{ScrollContext, ScrollKeyAndStartOffset}; use crate::search_job_placer::{Job, group_by, group_jobs_by_index_id}; use crate::search_response_rest::StorageRequestCount; @@ -572,7 +572,7 @@ async fn search_partial_hits_phase_with_scroll( mut search_request: SearchRequest, split_metadatas: &[SplitMetadata], cluster_client: &ClusterClient, - lambda_invoker: Option>, + lambda_invoker: Option>, ) -> crate::Result<(LeafSearchResponse, Option)> { let scroll_ttl_opt = get_scroll_ttl_duration(&search_request)?; @@ -742,7 +742,7 @@ pub(crate) async fn search_partial_hits_phase( search_request: &SearchRequest, split_metadatas: &[SplitMetadata], cluster_client: &ClusterClient, - lambda_invoker: Option>, + lambda_invoker: Option>, ) -> crate::Result { let leaf_search_responses: Vec = if is_metadata_count_request(search_request) { @@ -838,7 +838,7 @@ async fn execute_leaf_search_via_lambda( search_request: &SearchRequest, indexes_metas_for_leaf_search: &IndexesMetasForLeafSearch, split_metadatas: &[SplitMetadata], - lambda_invoker: &dyn RemoteFunctionInvoker, + lambda_invoker: &dyn LambdaLeafSearchInvoker, lambda_config: &quickwit_config::LambdaConfig, ) -> crate::Result> { let jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); @@ -1048,7 +1048,7 @@ async fn root_search_aux( search_request: SearchRequest, split_metadatas: Vec, cluster_client: &ClusterClient, - lambda_invoker: Option>, + lambda_invoker: Option>, ) -> crate::Result { debug!(split_metadatas = ?PrettySample::new(&split_metadatas, 5)); let (first_phase_result, scroll_key_and_start_offset_opt): ( @@ -1279,7 +1279,7 @@ pub async fn root_search( mut search_request: SearchRequest, mut metastore: MetastoreServiceClient, cluster_client: &ClusterClient, - lambda_invoker: Option>, + lambda_invoker: Option>, ) -> crate::Result { let start_instant = Instant::now(); diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index 851358ee965..b074557f730 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -33,13 +33,13 @@ use quickwit_storage::{ }; use tantivy::aggregation::AggregationLimitsGuard; +use crate::invoker::LambdaLeafSearchInvoker; use crate::leaf::multi_index_leaf_search; use crate::leaf_cache::{LeafSearchCache, PredicateCacheImpl}; use crate::list_fields::{leaf_list_fields, root_list_fields}; use crate::list_fields_cache::ListFieldsCache; use crate::list_terms::{leaf_list_terms, root_list_terms}; use crate::metrics_trackers::LeafSearchMetricsFuture; -use crate::remote_function::RemoteFunctionInvoker; use crate::root::fetch_docs_phase; use crate::scroll_context::{MiniKV, ScrollContext, ScrollKeyAndStartOffset}; use crate::search_permit_provider::SearchPermitProvider; @@ -53,7 +53,7 @@ pub struct SearchServiceImpl { cluster_client: ClusterClient, searcher_context: Arc, local_kv_store: MiniKV, - lambda_invoker: Option>, + lambda_invoker: Option>, } /// Trait representing a search service. @@ -143,7 +143,7 @@ impl SearchServiceImpl { storage_resolver: StorageResolver, cluster_client: ClusterClient, searcher_context: Arc, - lambda_invoker: Option>, + lambda_invoker: Option>, ) -> Self { SearchServiceImpl { metastore, diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 4f007ec255a..1733f73b628 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1019,22 +1019,9 @@ async fn setup_searcher( let search_job_placer = SearchJobPlacer::new(searcher_pool.clone()); // Initialize Lambda invoker if enabled - let lambda_invoker = if let Some(lambda_config) = &node_config.searcher_config.lambda { + let lambda_invoker_opt = if let Some(lambda_config) = &node_config.searcher_config.lambda { info!("initializing AWS Lambda invoker for leaf search"); - - // Auto-deploy Lambda function if configured - if let Some(deploy_config) = &lambda_config.auto_deploy { - info!("auto-deploying Lambda function"); - quickwit_lambda_client::deploy(&lambda_config.function_name, deploy_config) - .await - .context("failed to deploy lambda function") - .inspect_err(|err| error!(err=?err, "deploy lambda failed"))?; - } - - let invoker = quickwit_lambda_client::create_lambda_invoker(lambda_config) - .await - .context("failed to initialize AWS Lambda invoker")?; - + let invoker = quickwit_lambda_client::try_get_or_deploy_invoker(lambda_config).await?; Some(invoker) } else { None @@ -1045,7 +1032,7 @@ async fn setup_searcher( storage_resolver, search_job_placer.clone(), searcher_context, - lambda_invoker, + lambda_invoker_opt, ) .await?; let search_service_clone = search_service.clone(); @@ -1577,7 +1564,8 @@ mod tests { #[tokio::test] async fn test_setup_searcher() { let node_config = NodeConfig::for_test(); - let searcher_context = Arc::new(SearcherContext::new(SearcherConfig::default(), None)); + let searcher_context = + Arc::new(SearcherContext::new(SearcherConfig::default(), None, None)); let metastore = metastore_for_test(); let (change_stream, change_stream_tx) = ClusterChangeStream::new_unbounded(); let storage_resolver = StorageResolver::unconfigured(); From 93cd5b9b52cda0e6f37e5153bf74767b9d6c99e2 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Fri, 6 Feb 2026 14:48:21 +0100 Subject: [PATCH 21/22] Added unit tests --- quickwit/Cargo.lock | 35 +- quickwit/Cargo.toml | 1 + quickwit/quickwit-lambda-client/Cargo.toml | 15 +- quickwit/quickwit-lambda-client/src/deploy.rs | 376 +++++++++++++++++- quickwit/quickwit-serve/src/lib.rs | 2 +- 5 files changed, 413 insertions(+), 16 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index d6b76dc61b6..b0218648453 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -708,9 +708,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.7" +version = "1.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ee19095c7c4dda59f1697d028ce704c24b2d33c6718790c7f1d5a3015b4107c" +checksum = "52eec3db979d18cb807fc1070961cc51d87d069abe9ab57917769687368a8c6c" dependencies = [ "futures-util", "pin-project-lite", @@ -794,9 +794,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.5" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59e62db736db19c488966c8d787f52e6270be565727236fd5579eaa301e7bc4a" +checksum = "12fb0abf49ff0cab20fd31ac1215ed7ce0ea92286ba09e2854b42ba5cabe7525" dependencies = [ "aws-smithy-async", "aws-smithy-protocol-test", @@ -846,6 +846,18 @@ dependencies = [ "aws-smithy-types", ] +[[package]] +name = "aws-smithy-mocks" +version = "0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "662b7376269470b43978208ff2f2a233512219eb4f60ea08555649bca1bd0855" +dependencies = [ + "aws-smithy-http-client", + "aws-smithy-runtime-api", + "aws-smithy-types", + "http 1.4.0", +] + [[package]] name = "aws-smithy-observability" version = "0.1.5" @@ -857,9 +869,9 @@ dependencies = [ [[package]] name = "aws-smithy-protocol-test" -version = "0.63.7" +version = "0.63.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01317a9e3c5c06f1af35001ef0c873c1e34e458c20b2ee1eee0fb431e6dbb010" +checksum = "7090f3a3657e7c2c0331604d62baa20a9a9f765c3f4bf63ccf48ccba6b8b7240" dependencies = [ "assert-json-diff", "aws-smithy-runtime-api", @@ -911,9 +923,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.9.3" +version = "1.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab0d43d899f9e508300e587bf582ba54c27a452dd0a9ea294690669138ae14a2" +checksum = "49952c52f7eebb72ce2a754d3866cc0f87b97d2a46146b79f80f3a93fb2b3716" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -928,9 +940,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.5" +version = "1.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "905cb13a9895626d49cf2ced759b062d913834c7482c38e49557eac4e6193f01" +checksum = "3b3a26048eeab0ddeba4b4f9d51654c79af8c3b32357dc5f336cee85ab331c33" dependencies = [ "base64-simd", "bytes", @@ -7355,7 +7367,9 @@ dependencies = [ "async-trait", "aws-config", "aws-sdk-lambda", + "aws-smithy-mocks", "base64 0.22.1", + "bytesize", "hex", "once_cell", "prost 0.14.1", @@ -7364,6 +7378,7 @@ dependencies = [ "quickwit-lambda-server", "quickwit-proto", "quickwit-search", + "quickwit-storage", "serde_json", "sha1", "thiserror 2.0.17", diff --git a/quickwit/Cargo.toml b/quickwit/Cargo.toml index ec87cedacdf..a46d718ec4b 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -325,6 +325,7 @@ aws-sdk-s3 = "=1.62" aws-sdk-lambda = "1" aws-sdk-sqs = "1.91" aws-smithy-async = "1.2" +aws-smithy-mocks = "0.2" aws-smithy-http-client = { version = "1.1", features = ["default-client"] } aws-smithy-runtime = "1.9" aws-smithy-types = { version = "1.3", features = [ diff --git a/quickwit/quickwit-lambda-client/Cargo.toml b/quickwit/quickwit-lambda-client/Cargo.toml index b683006574c..375bff2bd3a 100644 --- a/quickwit/quickwit-lambda-client/Cargo.toml +++ b/quickwit/quickwit-lambda-client/Cargo.toml @@ -29,11 +29,18 @@ quickwit-lambda-server = { workspace = true } quickwit-proto = { workspace = true } quickwit-search = { workspace = true } +[dev-dependencies] +aws-smithy-mocks = { workspace = true } +aws-sdk-lambda = { workspace = true, features = ["test-util"] } +bytesize = { workspace = true } +tokio = { workspace = true, features = ["test-util", "macros"] } + +# Required for complicated reasons. quickwit-storage checks that we +# do use preserve order with serde. aws forces that feature. We disable +# the check by switching on its testsuite feature. +quickwit-storage = { workspace = true, features = ["testsuite"] } + [build-dependencies] hex = { workspace = true } sha1 = "0.10" ureq = { workspace = true } - -[features] -default = [] -testsuite = [] diff --git a/quickwit/quickwit-lambda-client/src/deploy.rs b/quickwit/quickwit-lambda-client/src/deploy.rs index ad9e2bd75c6..24319eb9ab7 100644 --- a/quickwit/quickwit-lambda-client/src/deploy.rs +++ b/quickwit/quickwit-lambda-client/src/deploy.rs @@ -183,7 +183,7 @@ async fn find_matching_version( let response = match request.send().await { Ok(resp) => resp, Err(SdkError::ServiceError(err)) if err.err().is_resource_not_found_exception() => { - debug!( + info!( function_name = %function_name, "Lambda function does not exist yet" ); @@ -598,3 +598,377 @@ fn build_tags() -> HashMap { tags.insert("managed_by".to_string(), "quickwit".to_string()); tags } + +#[cfg(test)] +mod tests { + use aws_sdk_lambda::operation::create_function::{CreateFunctionError, CreateFunctionOutput}; + use aws_sdk_lambda::operation::delete_function::DeleteFunctionOutput; + use aws_sdk_lambda::operation::get_function::GetFunctionOutput; + use aws_sdk_lambda::operation::list_versions_by_function::{ + ListVersionsByFunctionError, ListVersionsByFunctionOutput, + }; + use aws_sdk_lambda::operation::publish_version::PublishVersionOutput; + use aws_sdk_lambda::operation::update_function_code::UpdateFunctionCodeOutput; + use aws_sdk_lambda::types::FunctionConfiguration; + use aws_sdk_lambda::types::error::{ResourceConflictException, ResourceNotFoundException}; + use aws_smithy_mocks::{RuleMode, mock, mock_client}; + use bytesize::ByteSize; + + use super::*; + + fn make_version(version: &str, description: &str) -> FunctionConfiguration { + FunctionConfiguration::builder() + .version(version) + .description(description) + .build() + } + + fn test_deploy_config() -> LambdaDeployConfig { + LambdaDeployConfig { + execution_role_arn: "arn:aws:iam::123456789:role/test-role".to_string(), + memory_size: ByteSize::gib(5), + invocation_timeout_secs: 60, + } + } + + // --- find_matching_version tests --- + + #[tokio::test] + async fn test_find_matching_version_found() { + let target = "quickwit:test_version"; + let rule = mock!(aws_sdk_lambda::Client::list_versions_by_function).then_output(|| { + ListVersionsByFunctionOutput::builder() + .versions(make_version("$LATEST", "")) + .versions(make_version("1", "quickwit:old_version")) + .versions(make_version("7", "quickwit:test_version")) + .build() + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + + let matching_version_opt = find_matching_version(&client, "my-fn", target) + .await + .unwrap(); + assert_eq!(matching_version_opt, Some("7".to_string())); + } + + #[tokio::test] + async fn test_find_matching_version_not_found() { + let rule = mock!(aws_sdk_lambda::Client::list_versions_by_function).then_output(|| { + ListVersionsByFunctionOutput::builder() + .versions(make_version("$LATEST", "")) + .versions(make_version("1", "quickwit:other")) + .build() + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + + let result = find_matching_version(&client, "my-fn", "quickwit:no_match") + .await + .unwrap(); + assert_eq!(result, None); + } + + #[tokio::test] + async fn test_find_matching_version_function_does_not_exist() { + let rule = mock!(aws_sdk_lambda::Client::list_versions_by_function).then_error(|| { + ListVersionsByFunctionError::ResourceNotFoundException( + ResourceNotFoundException::builder().build(), + ) + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + + let result = find_matching_version(&client, "no-such-fn", "quickwit:x") + .await + .unwrap(); + assert_eq!(result, None); + } + + #[tokio::test] + async fn test_find_matching_version_skips_latest_even_if_description_matches() { + let rule = mock!(aws_sdk_lambda::Client::list_versions_by_function).then_output(|| { + ListVersionsByFunctionOutput::builder() + .versions(make_version("$LATEST", "quickwit:match")) + .build() + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + + let result = find_matching_version(&client, "my-fn", "quickwit:match") + .await + .unwrap(); + assert_eq!(result, None); + } + + // --- try_create_function tests --- + + #[tokio::test] + async fn test_try_create_function_success() { + let rule = mock!(aws_sdk_lambda::Client::create_function).then_output(|| { + CreateFunctionOutput::builder() + .version("1") + .function_name("my-fn") + .build() + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + let config = test_deploy_config(); + + let result = try_create_function(&client, "my-fn", &config) + .await + .unwrap(); + assert_eq!(result, Some("1".to_string())); + } + + #[tokio::test] + async fn test_try_create_function_already_exists() { + let rule = mock!(aws_sdk_lambda::Client::create_function).then_error(|| { + CreateFunctionError::ResourceConflictException( + ResourceConflictException::builder().build(), + ) + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + let config = test_deploy_config(); + + let result = try_create_function(&client, "my-fn", &config) + .await + .unwrap(); + assert_eq!(result, None); + } + + // --- deploy (update path) tests --- + + #[tokio::test] + async fn test_deploy_update_path() { + // create_function → conflict (function exists) + let create_rule = mock!(aws_sdk_lambda::Client::create_function).then_error(|| { + CreateFunctionError::ResourceConflictException( + ResourceConflictException::builder().build(), + ) + }); + // update_function_code → success with code_sha256 + let update_rule = mock!(aws_sdk_lambda::Client::update_function_code).then_output(|| { + UpdateFunctionCodeOutput::builder() + .code_sha256("abc123hash") + .build() + }); + // get_function → active and ready (for wait_for_function_ready) + let get_rule = mock!(aws_sdk_lambda::Client::get_function).then_output(|| { + GetFunctionOutput::builder() + .configuration( + FunctionConfiguration::builder() + .state(State::Active) + .last_update_status(LastUpdateStatus::Successful) + .build(), + ) + .build() + }); + // publish_version → success + let publish_rule = mock!(aws_sdk_lambda::Client::publish_version) + .then_output(|| PublishVersionOutput::builder().version("8").build()); + + let client = mock_client!( + aws_sdk_lambda, + RuleMode::MatchAny, + [&create_rule, &update_rule, &get_rule, &publish_rule] + ); + let config = test_deploy_config(); + + tokio::time::pause(); + let version = deploy_lambda_function_inner(&client, "my-fn", &config) + .await + .unwrap(); + assert_eq!(version, "8"); + } + + // --- wait_for_function_ready tests --- + + #[tokio::test] + async fn test_wait_for_function_ready_immediate() { + let rule = mock!(aws_sdk_lambda::Client::get_function).then_output(|| { + GetFunctionOutput::builder() + .configuration( + FunctionConfiguration::builder() + .state(State::Active) + .last_update_status(LastUpdateStatus::Successful) + .build(), + ) + .build() + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + + tokio::time::pause(); + wait_for_function_ready(&client, "my-fn").await.unwrap(); + } + + #[tokio::test] + async fn test_wait_for_function_ready_after_update_in_progress() { + let rule = mock!(aws_sdk_lambda::Client::get_function) + .sequence() + .output(|| { + GetFunctionOutput::builder() + .configuration( + FunctionConfiguration::builder() + .state(State::Active) + .last_update_status(LastUpdateStatus::InProgress) + .build(), + ) + .build() + }) + .output(|| { + GetFunctionOutput::builder() + .configuration( + FunctionConfiguration::builder() + .state(State::Active) + .last_update_status(LastUpdateStatus::Successful) + .build(), + ) + .build() + }) + .build(); + let client = mock_client!(aws_sdk_lambda, RuleMode::Sequential, [&rule]); + + tokio::time::pause(); + wait_for_function_ready(&client, "my-fn").await.unwrap(); + assert_eq!(rule.num_calls(), 2); + } + + #[tokio::test] + async fn test_wait_for_function_ready_fails_on_failed_state() { + let rule = mock!(aws_sdk_lambda::Client::get_function).then_output(|| { + GetFunctionOutput::builder() + .configuration( + FunctionConfiguration::builder() + .state(State::Failed) + .state_reason("Something broke") + .build(), + ) + .build() + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + + tokio::time::pause(); + let err = wait_for_function_ready(&client, "my-fn").await.unwrap_err(); + assert!( + err.to_string().contains("Failed state"), + "unexpected error: {}", + err + ); + } + + #[tokio::test] + async fn test_wait_for_function_ready_fails_on_last_update_failed() { + let rule = mock!(aws_sdk_lambda::Client::get_function).then_output(|| { + GetFunctionOutput::builder() + .configuration( + FunctionConfiguration::builder() + .state(State::Active) + .last_update_status(LastUpdateStatus::Failed) + .last_update_status_reason("Update broke") + .build(), + ) + .build() + }); + let client = mock_client!(aws_sdk_lambda, [&rule]); + + tokio::time::pause(); + let err = wait_for_function_ready(&client, "my-fn").await.unwrap_err(); + assert!( + err.to_string().contains("last update failed"), + "unexpected error: {}", + err + ); + } + + // --- garbage_collect_old_versions tests --- + + #[tokio::test] + async fn test_gc_deletes_old_versions_keeps_recent() { + // 8 quickwit versions (1..=8) + $LATEST + one non-quickwit version + let list_rule = + mock!(aws_sdk_lambda::Client::list_versions_by_function).then_output(|| { + let mut builder = ListVersionsByFunctionOutput::builder() + .versions(make_version("$LATEST", "")) + .versions(make_version("99", "not-quickwit")); + for i in 1..=8 { + builder = builder + .versions(make_version(&i.to_string(), &format!("quickwit:ver_{}", i))); + } + builder.build() + }); + + let delete_rule = mock!(aws_sdk_lambda::Client::delete_function) + .then_output(|| DeleteFunctionOutput::builder().build()); + + let client = mock_client!( + aws_sdk_lambda, + RuleMode::MatchAny, + [&list_rule, &delete_rule] + ); + + // Current version is "7", so keep 7 + the 5 most recent (4,5,6,7,8). + // Should delete versions 1, 2, 3. + garbage_collect_old_versions(&client, "my-fn", "7") + .await + .unwrap(); + + assert_eq!(delete_rule.num_calls(), 3); + } + + #[tokio::test] + async fn test_gc_nothing_to_delete() { + // Only 3 quickwit versions — below the GC_KEEP_RECENT_VERSIONS threshold. + let list_rule = + mock!(aws_sdk_lambda::Client::list_versions_by_function).then_output(|| { + ListVersionsByFunctionOutput::builder() + .versions(make_version("$LATEST", "")) + .versions(make_version("1", "quickwit:v1")) + .versions(make_version("2", "quickwit:v2")) + .versions(make_version("3", "quickwit:v3")) + .build() + }); + + let delete_rule = mock!(aws_sdk_lambda::Client::delete_function) + .then_output(|| DeleteFunctionOutput::builder().build()); + + let client = mock_client!( + aws_sdk_lambda, + RuleMode::MatchAny, + [&list_rule, &delete_rule] + ); + + garbage_collect_old_versions(&client, "my-fn", "3") + .await + .unwrap(); + + assert_eq!(delete_rule.num_calls(), 0); + } + + #[tokio::test] + async fn test_gc_does_not_delete_current_version() { + // 7 quickwit versions, current is "1" (the oldest). + // Without the current-version guard, version 1 would be deleted. + let list_rule = + mock!(aws_sdk_lambda::Client::list_versions_by_function).then_output(|| { + let mut builder = + ListVersionsByFunctionOutput::builder().versions(make_version("$LATEST", "")); + for i in 1..=7 { + builder = builder + .versions(make_version(&i.to_string(), &format!("quickwit:ver_{}", i))); + } + builder.build() + }); + + let delete_rule = mock!(aws_sdk_lambda::Client::delete_function) + .then_output(|| DeleteFunctionOutput::builder().build()); + + let client = mock_client!( + aws_sdk_lambda, + RuleMode::MatchAny, + [&list_rule, &delete_rule] + ); + + // Current version is "1". Without guard: would delete 1,2. With guard: only deletes 2. + garbage_collect_old_versions(&client, "my-fn", "1") + .await + .unwrap(); + + assert_eq!(delete_rule.num_calls(), 1); + } +} diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 1733f73b628..cfe69b5cc96 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -1565,7 +1565,7 @@ mod tests { async fn test_setup_searcher() { let node_config = NodeConfig::for_test(); let searcher_context = - Arc::new(SearcherContext::new(SearcherConfig::default(), None, None)); + Arc::new(SearcherContext::new(SearcherConfig::default(), None)); let metastore = metastore_for_test(); let (change_stream, change_stream_tx) = ClusterChangeStream::new_unbounded(); let storage_resolver = StorageResolver::unconfigured(); From fd9564f113d04bbfb2063868ba71bf06ddad689d Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Fri, 6 Feb 2026 17:47:53 +0100 Subject: [PATCH 22/22] Moving lambda execution to the leaf. --- .../quickwit-config/src/node_config/mod.rs | 9 + .../quickwit-lambda-client/src/invoker.rs | 12 +- .../quickwit-lambda-server/src/context.rs | 2 +- .../quickwit-lambda-server/src/handler.rs | 85 ++++++-- .../protos/quickwit/search.proto | 19 +- .../src/codegen/quickwit/quickwit.search.rs | 44 +---- quickwit/quickwit-search/src/invoker.rs | 4 +- quickwit/quickwit-search/src/leaf.rs | 181 +++++++++++++++--- quickwit/quickwit-search/src/lib.rs | 6 +- quickwit/quickwit-search/src/root.rs | 109 +---------- .../quickwit-search/src/scroll_context.rs | 1 - .../src/search_permit_provider.rs | 70 ++++++- quickwit/quickwit-search/src/service.rs | 15 +- quickwit/quickwit-search/src/tests.rs | 4 +- .../src/elasticsearch_api/rest_handler.rs | 1 - quickwit/quickwit-serve/src/lib.rs | 22 +-- .../src/search_api/rest_handler.rs | 1 - 17 files changed, 351 insertions(+), 234 deletions(-) diff --git a/quickwit/quickwit-config/src/node_config/mod.rs b/quickwit/quickwit-config/src/node_config/mod.rs index 13232158cad..ea0689299ad 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -320,6 +320,11 @@ pub struct LambdaConfig { /// Maximum number of splits per Lambda invocation. #[serde(default = "LambdaConfig::default_max_splits_per_invocation")] pub max_splits_per_invocation: usize, + /// When the number of pending split searches exceeds this threshold, + /// new splits are offloaded to Lambda instead of being queued locally. + /// A value of 0 disables offloading (all splits are processed locally). + #[serde(default = "LambdaConfig::default_offload_threshold")] + pub offload_threshold: usize, /// Auto-deploy configuration. If set, Quickwit will automatically deploy /// the Lambda function at startup. /// If deploying a lambda fails, Quickwit will log an error and fail. @@ -356,6 +361,7 @@ impl Default for LambdaConfig { Self { function_name: Self::default_function_name(), max_splits_per_invocation: Self::default_max_splits_per_invocation(), + offload_threshold: Self::default_offload_threshold(), auto_deploy: None, } } @@ -368,6 +374,9 @@ impl LambdaConfig { fn default_max_splits_per_invocation() -> usize { 10 } + fn default_offload_threshold() -> usize { + 20 + } } #[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] diff --git a/quickwit/quickwit-lambda-client/src/invoker.rs b/quickwit/quickwit-lambda-client/src/invoker.rs index f078784ec5f..fd3c35b8fe0 100644 --- a/quickwit/quickwit-lambda-client/src/invoker.rs +++ b/quickwit/quickwit-lambda-client/src/invoker.rs @@ -21,7 +21,7 @@ use aws_sdk_lambda::types::InvocationType; use base64::prelude::*; use prost::Message; use quickwit_lambda_server::{LeafSearchPayload, LeafSearchResponsePayload}; -use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; +use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse, LeafSearchResponses}; use quickwit_search::{LambdaLeafSearchInvoker, SearchError}; use tracing::{debug, info, instrument}; @@ -91,7 +91,7 @@ impl LambdaLeafSearchInvoker for AwsLambdaInvoker { async fn invoke_leaf_search( &self, request: LeafSearchRequest, - ) -> Result { + ) -> Result, SearchError> { let start = std::time::Instant::now(); let result = self.invoke_leaf_search_inner(request).await; @@ -115,7 +115,7 @@ impl AwsLambdaInvoker { async fn invoke_leaf_search_inner( &self, request: LeafSearchRequest, - ) -> Result { + ) -> Result, SearchError> { // Serialize request to protobuf bytes, then base64 encode let request_bytes = request.encode_to_vec(); let payload = LeafSearchPayload { @@ -178,14 +178,14 @@ impl AwsLambdaInvoker { .decode(&lambda_response.payload) .map_err(|e| SearchError::Internal(format!("Base64 decode error: {}", e)))?; - let leaf_response = LeafSearchResponse::decode(&response_bytes[..]) + let leaf_responses = LeafSearchResponses::decode(&response_bytes[..]) .map_err(|e| SearchError::Internal(format!("Protobuf decode error: {}", e)))?; debug!( - num_hits = leaf_response.num_hits, + num_responses = leaf_responses.responses.len(), "Lambda invocation completed" ); - Ok(leaf_response) + Ok(leaf_responses.responses) } } diff --git a/quickwit/quickwit-lambda-server/src/context.rs b/quickwit/quickwit-lambda-server/src/context.rs index 5381994aedb..eed01175a9f 100644 --- a/quickwit/quickwit-lambda-server/src/context.rs +++ b/quickwit/quickwit-lambda-server/src/context.rs @@ -34,7 +34,7 @@ impl LambdaSearcherContext { let config = LambdaSearcherConfig::try_from_env()?; let searcher_config = create_searcher_config(&config); - let searcher_context = Arc::new(SearcherContext::new(searcher_config, None)); + let searcher_context = Arc::new(SearcherContext::new(searcher_config, None, None)); let storage_resolver = StorageResolver::configured(&Default::default()); Ok(Self { diff --git a/quickwit/quickwit-lambda-server/src/handler.rs b/quickwit/quickwit-lambda-server/src/handler.rs index 24231916615..bc346f89cce 100644 --- a/quickwit/quickwit-lambda-server/src/handler.rs +++ b/quickwit/quickwit-lambda-server/src/handler.rs @@ -14,9 +14,12 @@ use base64::prelude::*; use prost::Message; -use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; +use quickwit_proto::search::{ + LeafSearchRequest, LeafSearchResponse, LeafSearchResponses, SplitIdAndFooterOffsets, +}; use quickwit_search::leaf::multi_index_leaf_search; use serde::{Deserialize, Serialize}; +use tokio::task::JoinSet; use tracing::{info, instrument}; use crate::context::LambdaSearcherContext; @@ -32,11 +35,14 @@ pub struct LeafSearchPayload { /// Response from leaf search Lambda invocation. #[derive(Debug, Serialize, Deserialize)] pub struct LeafSearchResponsePayload { - /// Base64-encoded serialized LeafSearchResponse protobuf. + /// Base64-encoded serialized `LeafSearchResponses` protobuf (one per split). pub payload: String, } /// Handle a leaf search request in Lambda. +/// +/// Returns one `LeafSearchResponse` per split. Each split is processed +/// independently so that the caller can cache and merge results individually. #[instrument(skip(ctx), fields(request_id))] pub async fn handle_leaf_search( event: LeafSearchPayload, @@ -50,30 +56,73 @@ pub async fn handle_leaf_search( // Deserialize LeafSearchRequest let leaf_search_request = LeafSearchRequest::decode(&request_bytes[..])?; - let num_splits: usize = leaf_search_request + let all_splits: Vec<(usize, SplitIdAndFooterOffsets)> = leaf_search_request .leaf_requests .iter() - .map(|leaf_request_ref| leaf_request_ref.split_offsets.len()) - .sum(); + .enumerate() + .flat_map(|(leaf_req_idx, leaf_request_ref)| { + leaf_request_ref + .split_offsets + .iter() + .cloned() + .map(move |split_id_and_footer_offsets| (leaf_req_idx, split_id_and_footer_offsets)) + }) + .collect(); - info!(num_splits, "processing leaf search request"); + let num_splits = all_splits.len(); + info!(num_splits, "processing leaf search request (per-split)"); - // Execute leaf search - let leaf_search_response = multi_index_leaf_search( - ctx.searcher_context.clone(), - leaf_search_request, - &ctx.storage_resolver, - ) - .await?; + // Process each split in parallel. The SearchPermitProvider inside + // SearcherContext gates concurrency based on memory budget. + let mut join_set = JoinSet::new(); + for (split_idx, (leaf_req_idx, split)) in all_splits.into_iter().enumerate() { + let leaf_request_ref = &leaf_search_request.leaf_requests[leaf_req_idx]; + let single_split_request = LeafSearchRequest { + search_request: leaf_search_request.search_request.clone(), + doc_mappers: leaf_search_request.doc_mappers.clone(), + index_uris: leaf_search_request.index_uris.clone(), + leaf_requests: vec![quickwit_proto::search::LeafRequestRef { + index_uri_ord: leaf_request_ref.index_uri_ord, + doc_mapper_ord: leaf_request_ref.doc_mapper_ord, + split_offsets: vec![split], + }], + }; + + let searcher_context = ctx.searcher_context.clone(); + let storage_resolver = ctx.storage_resolver.clone(); + join_set.spawn(async move { + let response = multi_index_leaf_search( + searcher_context, + single_split_request, + &storage_resolver, + ) + .await; + (split_idx, response) + }); + } + + // Collect results, preserving split order. + let mut indexed_responses: Vec<(usize, LeafSearchResponse)> = + Vec::with_capacity(num_splits); + while let Some(join_result) = join_set.join_next().await { + let (split_idx, search_result) = join_result + .map_err(|e| LambdaError::Internal(format!("split search task failed: {e}")))?; + let response = search_result + .map_err(|e| LambdaError::Internal(format!("leaf search failed: {e}")))?; + indexed_responses.push((split_idx, response)); + } + indexed_responses.sort_by_key(|(idx, _)| *idx); + let responses: Vec = + indexed_responses.into_iter().map(|(_, r)| r).collect(); info!( - num_hits = leaf_search_response.num_hits, - num_successful_splits = leaf_search_response.num_successful_splits, - "leaf search completed" + num_responses = responses.len(), + "leaf search completed (per-split)" ); - // Serialize response - let response_bytes = leaf_search_response.encode_to_vec(); + // Serialize as LeafSearchResponses wrapper + let wrapper = LeafSearchResponses { responses }; + let response_bytes = wrapper.encode_to_vec(); let payload = BASE64_STANDARD.encode(&response_bytes); Ok(LeafSearchResponsePayload { payload }) diff --git a/quickwit/quickwit-proto/protos/quickwit/search.proto b/quickwit/quickwit-proto/protos/quickwit/search.proto index dd18d741394..33eb2fdd01c 100644 --- a/quickwit/quickwit-proto/protos/quickwit/search.proto +++ b/quickwit/quickwit-proto/protos/quickwit/search.proto @@ -182,14 +182,6 @@ message ListFields { } // -- Search ------------------- -// Execution mode for leaf search operations. -enum ExecutionMode { - // Default: use gRPC to searcher nodes (existing behavior). - EXECUTION_MODE_GRPC = 0; - // Execute leaf search via remote serverless functions (e.g., AWS Lambda). - EXECUTION_MODE_REMOTE_FUNCTION = 1; -} - message SearchRequest { // Index ID patterns repeated string index_id_patterns = 1; @@ -255,11 +247,6 @@ message SearchRequest { // When an exact index ID is provided (not a pattern), the query fails only if // that index is not found and this parameter is set to `false`. bool ignore_missing_indexes = 18; - - // Execution mode for leaf search operations. - // When set to EXECUTION_MODE_REMOTE_FUNCTION, leaf search is executed via - // serverless functions (e.g., AWS Lambda) instead of gRPC to searcher nodes. - ExecutionMode execution_mode = 19; } enum CountHits { @@ -506,6 +493,12 @@ message LeafSearchResponse { ResourceStats resource_stats = 8; } +// Wrapper for multiple LeafSearchResponse messages, used by Lambda to return +// per-split results. +message LeafSearchResponses { + repeated LeafSearchResponse responses = 1; +} + message SnippetRequest { repeated string snippet_fields = 1; string query_ast_resolved = 2; diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs index 9ce1addf223..f0d9ad3d434 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs @@ -185,11 +185,6 @@ pub struct SearchRequest { /// that index is not found and this parameter is set to `false`. #[prost(bool, tag = "18")] pub ignore_missing_indexes: bool, - /// Execution mode for leaf search operations. - /// When set to EXECUTION_MODE_REMOTE_FUNCTION, leaf search is executed via - /// serverless functions (e.g., AWS Lambda) instead of gRPC to searcher nodes. - #[prost(enumeration = "ExecutionMode", tag = "19")] - pub execution_mode: i32, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] @@ -461,6 +456,14 @@ pub struct LeafSearchResponse { #[prost(message, optional, tag = "8")] pub resource_stats: ::core::option::Option, } +/// Wrapper for multiple LeafSearchResponse messages, used by Lambda to return +/// per-split results. +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct LeafSearchResponses { + #[prost(message, repeated, tag = "1")] + pub responses: ::prost::alloc::vec::Vec, +} #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] pub struct SnippetRequest { @@ -620,37 +623,6 @@ impl ListFieldType { } } } -/// Execution mode for leaf search operations. -#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] -#[serde(rename_all = "snake_case")] -#[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] -#[repr(i32)] -pub enum ExecutionMode { - /// Default: use gRPC to searcher nodes (existing behavior). - Grpc = 0, - /// Execute leaf search via remote serverless functions (e.g., AWS Lambda). - RemoteFunction = 1, -} -impl ExecutionMode { - /// String value of the enum field names used in the ProtoBuf definition. - /// - /// The values are not transformed in any way and thus are considered stable - /// (if the ProtoBuf definition does not change) and safe for programmatic use. - pub fn as_str_name(&self) -> &'static str { - match self { - Self::Grpc => "EXECUTION_MODE_GRPC", - Self::RemoteFunction => "EXECUTION_MODE_REMOTE_FUNCTION", - } - } - /// Creates an enum from field names used in the ProtoBuf definition. - pub fn from_str_name(value: &str) -> ::core::option::Option { - match value { - "EXECUTION_MODE_GRPC" => Some(Self::Grpc), - "EXECUTION_MODE_REMOTE_FUNCTION" => Some(Self::RemoteFunction), - _ => None, - } - } -} #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[serde(rename_all = "snake_case")] #[derive(Clone, Copy, Debug, PartialEq, Eq, Hash, PartialOrd, Ord, ::prost::Enumeration)] diff --git a/quickwit/quickwit-search/src/invoker.rs b/quickwit/quickwit-search/src/invoker.rs index 7ee9ceb1c09..f160d88019f 100644 --- a/quickwit/quickwit-search/src/invoker.rs +++ b/quickwit/quickwit-search/src/invoker.rs @@ -26,8 +26,10 @@ use crate::SearchError; #[async_trait] pub trait LambdaLeafSearchInvoker: Send + Sync + 'static { /// Invoke the remote function with a LeafSearchRequest. + /// + /// Returns one `LeafSearchResponse` per split in the request. async fn invoke_leaf_search( &self, request: LeafSearchRequest, - ) -> Result; + ) -> Result, SearchError>; } diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index dd2dcb1e67c..f6b93a82d8c 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -1379,20 +1379,93 @@ pub async fn single_doc_mapping_leaf_search( let incremental_merge_collector = IncrementalCollector::new(merge_collector); let incremental_merge_collector = Arc::new(Mutex::new(incremental_merge_collector)); - // We acquire all of the leaf search permits to make sure our single split search tasks - // do no interleave with other leaf search requests. - let permit_sizes = split_with_req.iter().map(|(split, _)| { - compute_initial_memory_allocation( - split, - searcher_context - .searcher_config - .warmup_single_split_initial_allocation, - ) - }); - let permit_futures = searcher_context - .search_permit_provider - .get_permits(permit_sizes) - .await; + // Step 1: Check cache for each split before acquiring permits. + let mut uncached_splits: Vec<(SplitIdAndFooterOffsets, SearchRequest)> = + Vec::with_capacity(split_with_req.len()); + for (split, search_request) in split_with_req { + let mut rewritten_request = search_request.clone(); + rewrite_request( + &mut rewritten_request, + &split, + doc_mapper.timestamp_field_name(), + ); + if let Some(cached_response) = searcher_context + .leaf_search_cache + .get(split.clone(), rewritten_request) + { + incremental_merge_collector + .lock() + .unwrap() + .add_result(cached_response) + .ok(); + } else { + uncached_splits.push((split, search_request)); + } + } + + if uncached_splits.is_empty() { + let incremental_merge_collector = match Arc::try_unwrap(incremental_merge_collector) { + Ok(filter_merger) => filter_merger.into_inner().unwrap(), + Err(filter_merger) => filter_merger.lock().unwrap().clone(), + }; + return crate::search_thread_pool() + .run_cpu_intensive(|| incremental_merge_collector.finalize().map_err(Into::into)) + .instrument(info_span!("incremental_merge_intermediate")) + .await + .context("failed to merge split search responses")?; + } + + // Step 2: Determine which uncached splits to process locally vs offload. + let lambda_available = searcher_context.lambda_invoker.is_some() + && searcher_context.searcher_config.lambda.is_some(); + + let offload_threshold = if lambda_available { + searcher_context + .searcher_config + .lambda + .as_ref() + .unwrap() + .offload_threshold + } else { + 0 + }; + + let permit_sizes: Vec = uncached_splits + .iter() + .map(|(split, _)| { + compute_initial_memory_allocation( + split, + searcher_context + .searcher_config + .warmup_single_split_initial_allocation, + ) + }) + .collect(); + + let (local_splits_with_permits, offloaded_splits) = if lambda_available + && offload_threshold > 0 + { + let partition = searcher_context + .search_permit_provider + .get_permits_with_offload(permit_sizes, offload_threshold) + .await; + (partition.local, partition.offloaded) + } else { + let permit_futures = searcher_context + .search_permit_provider + .get_permits(permit_sizes) + .await; + let local: Vec<(usize, _)> = permit_futures.into_iter().enumerate().collect(); + (local, Vec::new()) + }; + + if !offloaded_splits.is_empty() { + info!( + num_local = local_splits_with_permits.len(), + num_offloaded = offloaded_splits.len(), + "partitioned splits between local and Lambda" + ); + } let leaf_search_context = Arc::new(LeafSearchContext { searcher_context: searcher_context.clone(), @@ -1402,17 +1475,17 @@ pub async fn single_doc_mapping_leaf_search( split_filter: split_filter.clone(), }); + // Step 3: Spawn local split search tasks. let mut split_search_futures = JoinSet::new(); - let mut task_id_to_split_id_map = HashMap::with_capacity(split_with_req.len()); - for ((split, search_request), permit_fut) in - split_with_req.into_iter().zip(permit_futures.into_iter()) - { + let mut task_id_to_split_id_map = HashMap::with_capacity(local_splits_with_permits.len()); + for (idx, permit_fut) in local_splits_with_permits { + let (split, search_request) = &uncached_splits[idx]; let leaf_split_search_permit = permit_fut .instrument(info_span!("waiting_for_leaf_search_split_semaphore")) .await; let Some(simplified_search_request) = - simplify_search_request(search_request, &split, &split_filter) + simplify_search_request(search_request.clone(), split, &split_filter) else { let mut leaf_search_state_guard = SplitSearchStateGuard::new(leaf_search_context.split_outcome_counters.clone()); @@ -1425,7 +1498,7 @@ pub async fn single_doc_mapping_leaf_search( simplified_search_request, leaf_search_context.clone(), index_storage.clone(), - split, + split.clone(), leaf_split_search_permit, aggregations_limits.clone(), ) @@ -1434,15 +1507,75 @@ pub async fn single_doc_mapping_leaf_search( task_id_to_split_id_map.insert(handle.id(), split_id); } - // TODO we could cancel running splits when !run_all_splits and the running split can no - // longer give better results after some other split answered. + // Step 4: Offload splits to Lambda. + if !offloaded_splits.is_empty() { + let lambda_invoker = searcher_context.lambda_invoker.as_ref().unwrap(); + let lambda_config = searcher_context.searcher_config.lambda.as_ref().unwrap(); + let batch_size = lambda_config.max_splits_per_invocation; + + let doc_mapper_str = serde_json::to_string(doc_mapper.as_ref()) + .map_err(|e| SearchError::Internal(format!("failed to serialize doc mapper: {e}")))?; + + // Build LeafSearchRequest with offloaded splits in batches. + let offloaded_split_offsets: Vec = offloaded_splits + .iter() + .map(|&idx| uncached_splits[idx].0.clone()) + .collect(); + + let index_uri = index_storage.uri().to_string(); + + let mut search_request_for_leaf = (*request).clone(); + search_request_for_leaf.start_offset = 0; + search_request_for_leaf.max_hits += request.start_offset; + + let mut lambda_tasks = Vec::new(); + for chunk in offloaded_split_offsets.chunks(batch_size) { + let leaf_request = LeafSearchRequest { + search_request: Some(search_request_for_leaf.clone()), + doc_mappers: vec![doc_mapper_str.clone()], + index_uris: vec![index_uri.clone()], + leaf_requests: vec![quickwit_proto::search::LeafRequestRef { + index_uri_ord: 0, + doc_mapper_ord: 0, + split_offsets: chunk.to_vec(), + }], + }; + let invoker = lambda_invoker.clone(); + lambda_tasks.push(async move { invoker.invoke_leaf_search(leaf_request).await }); + } + + let lambda_results = try_join_all(lambda_tasks).await; + match lambda_results { + Ok(batch_results) => { + let mut locked = incremental_merge_collector.lock().unwrap(); + for per_split_results in batch_results { + for response in per_split_results { + if let Err(err) = locked.add_result(response) { + error!(error = %err, "failed to add Lambda result to collector"); + } + } + } + } + Err(err) => { + error!(error = %err, "Lambda invocation failed for offloaded splits"); + let mut locked = incremental_merge_collector.lock().unwrap(); + for &idx in &offloaded_splits { + locked.add_failed_split(SplitSearchError { + split_id: uncached_splits[idx].0.split_id.clone(), + error: format!("Lambda invocation error: {err}"), + retryable_error: true, + }); + } + } + } + } + + // Step 5: Await all local tasks. let mut split_search_join_errors: Vec<(String, JoinError)> = Vec::new(); while let Some(leaf_search_join_result) = split_search_futures.join_next().await { - // splits that did not panic were already added to the collector if let Err(join_error) = leaf_search_join_result { if join_error.is_cancelled() { - // An explicit task cancellation is not an error. continue; } let split_id = task_id_to_split_id_map.get(&join_error.id()).unwrap(); diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 96721d83adb..89b45a69014 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -264,7 +264,6 @@ pub async fn start_searcher_service( storage_resolver: StorageResolver, search_job_placer: SearchJobPlacer, searcher_context: Arc, - lambda_invoker: Option>, ) -> anyhow::Result> { let cluster_client = ClusterClient::new(search_job_placer); let search_service = Arc::new(SearchServiceImpl::new( @@ -272,7 +271,6 @@ pub async fn start_searcher_service( storage_resolver, cluster_client, searcher_context, - lambda_invoker, )); Ok(search_service) } @@ -289,13 +287,12 @@ pub async fn single_node_search( let search_job_placer = SearchJobPlacer::new(searcher_pool.clone()); let cluster_client = ClusterClient::new(search_job_placer); let searcher_config = SearcherConfig::default(); - let searcher_context = Arc::new(SearcherContext::new(searcher_config, None)); + let searcher_context = Arc::new(SearcherContext::new(searcher_config, None, None)); let search_service = Arc::new(SearchServiceImpl::new( metastore.clone(), storage_resolver, cluster_client.clone(), searcher_context.clone(), - None, // No Lambda for single node search )); let search_service_client = SearchServiceClient::from_service(search_service.clone(), socket_addr); @@ -305,7 +302,6 @@ pub async fn single_node_search( search_request, metastore, &cluster_client, - None, // No Lambda for single node search ) .await } diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index 150c86418eb..ed292523ea0 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -14,7 +14,7 @@ use std::collections::{HashMap, HashSet}; use std::sync::atomic::{AtomicU64, Ordering}; -use std::sync::{Arc, OnceLock}; +use std::sync::OnceLock; use std::time::{Duration, Instant}; use anyhow::Context; @@ -31,10 +31,9 @@ use quickwit_proto::metastore::{ ListIndexesMetadataRequest, MetastoreService, MetastoreServiceClient, }; use quickwit_proto::search::{ - ExecutionMode, FetchDocsRequest, FetchDocsResponse, Hit, LeafHit, LeafRequestRef, - LeafSearchRequest, LeafSearchResponse, PartialHit, SearchPlanResponse, SearchRequest, - SearchResponse, SnippetRequest, SortDatetimeFormat, SortField, SortValue, - SplitIdAndFooterOffsets, + FetchDocsRequest, FetchDocsResponse, Hit, LeafHit, LeafRequestRef, LeafSearchRequest, + LeafSearchResponse, PartialHit, SearchPlanResponse, SearchRequest, SearchResponse, + SnippetRequest, SortDatetimeFormat, SortField, SortValue, SplitIdAndFooterOffsets, }; use quickwit_proto::types::{IndexUid, SplitId}; use quickwit_query::query_ast::{ @@ -50,7 +49,6 @@ use tracing::{debug, error, info, info_span, instrument}; use crate::cluster_client::ClusterClient; use crate::collector::{QuickwitAggregations, make_merge_collector}; -use crate::invoker::LambdaLeafSearchInvoker; use crate::metrics_trackers::{RootSearchMetricsFuture, RootSearchMetricsStep}; use crate::scroll_context::{ScrollContext, ScrollKeyAndStartOffset}; use crate::search_job_placer::{Job, group_by, group_jobs_by_index_id}; @@ -374,8 +372,6 @@ fn simplify_search_request_for_scroll_api(req: &SearchRequest) -> crate::Result< // to recompute it afterward. count_hits: quickwit_proto::search::CountHits::Underestimate as i32, ignore_missing_indexes: req.ignore_missing_indexes, - // Scroll requests don't support Lambda execution - execution_mode: ExecutionMode::Grpc as i32, }) } @@ -572,7 +568,6 @@ async fn search_partial_hits_phase_with_scroll( mut search_request: SearchRequest, split_metadatas: &[SplitMetadata], cluster_client: &ClusterClient, - lambda_invoker: Option>, ) -> crate::Result<(LeafSearchResponse, Option)> { let scroll_ttl_opt = get_scroll_ttl_duration(&search_request)?; @@ -591,7 +586,6 @@ async fn search_partial_hits_phase_with_scroll( &search_request, split_metadatas, cluster_client, - lambda_invoker.clone(), ) .await?; let cached_partial_hits = leaf_search_resp.partial_hits.clone(); @@ -637,7 +631,6 @@ async fn search_partial_hits_phase_with_scroll( &search_request, split_metadatas, cluster_client, - lambda_invoker, ) .await?; Ok((leaf_search_resp, None)) @@ -742,28 +735,10 @@ pub(crate) async fn search_partial_hits_phase( search_request: &SearchRequest, split_metadatas: &[SplitMetadata], cluster_client: &ClusterClient, - lambda_invoker: Option>, ) -> crate::Result { let leaf_search_responses: Vec = if is_metadata_count_request(search_request) { get_count_from_metadata(split_metadatas) - } else if search_request.execution_mode == ExecutionMode::RemoteFunction as i32 - && lambda_invoker.is_some() - && searcher_context.searcher_config.lambda.is_some() - { - // Execute via Lambda - info!( - num_splits = split_metadatas.len(), - "Starting lambda execution for leaf search" - ); - execute_leaf_search_via_lambda( - search_request, - indexes_metas_for_leaf_search, - split_metadatas, - lambda_invoker.as_ref().unwrap().as_ref(), - searcher_context.searcher_config.lambda.as_ref().unwrap(), - ) - .await? } else { let jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); let assigned_leaf_search_jobs = cluster_client @@ -830,54 +805,6 @@ pub(crate) async fn search_partial_hits_phase( Ok(leaf_search_response) } -/// Execute leaf search operations via remote serverless functions (e.g., AWS Lambda). -/// -/// This function batches splits into groups and invokes Lambda functions in parallel. -/// The batch size is controlled by `LambdaConfig::max_splits_per_invocation`. -async fn execute_leaf_search_via_lambda( - search_request: &SearchRequest, - indexes_metas_for_leaf_search: &IndexesMetasForLeafSearch, - split_metadatas: &[SplitMetadata], - lambda_invoker: &dyn LambdaLeafSearchInvoker, - lambda_config: &quickwit_config::LambdaConfig, -) -> crate::Result> { - let jobs: Vec = split_metadatas.iter().map(SearchJob::from).collect(); - - // Batch splits for Lambda invocations - let batch_size = lambda_config.max_splits_per_invocation; - let batches: Vec> = jobs - .chunks(batch_size) - .map(|chunk| chunk.to_vec()) - .collect(); - - debug!( - num_splits = split_metadatas.len(), - num_batches = batches.len(), - batch_size, - "Executing leaf search via Lambda" - ); - - // Invoke Lambda functions in parallel for each batch - let lambda_tasks: Vec<_> = batches - .into_iter() - .map(|batch_jobs| { - let request = - jobs_to_leaf_request(search_request, indexes_metas_for_leaf_search, batch_jobs); - async move { - let leaf_request = request?; - let response = lambda_invoker.invoke_leaf_search(leaf_request).await; - if let Err(err) = &response { - error!(error = %err, "Lambda invocation failed"); - } - response - } - }) - .collect(); - - let responses = try_join_all(lambda_tasks).await?; - Ok(responses) -} - pub(crate) fn get_snippet_request(search_request: &SearchRequest) -> Option { if search_request.snippet_fields.is_empty() { return None; @@ -1048,7 +975,6 @@ async fn root_search_aux( search_request: SearchRequest, split_metadatas: Vec, cluster_client: &ClusterClient, - lambda_invoker: Option>, ) -> crate::Result { debug!(split_metadatas = ?PrettySample::new(&split_metadatas, 5)); let (first_phase_result, scroll_key_and_start_offset_opt): ( @@ -1060,7 +986,6 @@ async fn root_search_aux( search_request.clone(), &split_metadatas[..], cluster_client, - lambda_invoker, ) .await?; @@ -1279,7 +1204,6 @@ pub async fn root_search( mut search_request: SearchRequest, mut metastore: MetastoreServiceClient, cluster_client: &ClusterClient, - lambda_invoker: Option>, ) -> crate::Result { let start_instant = Instant::now(); @@ -1329,7 +1253,6 @@ pub async fn root_search( search_request, split_metadatas, cluster_client, - lambda_invoker, ), is_success: None, step: RootSearchMetricsStep::Exec { @@ -2744,7 +2667,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -2815,7 +2737,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -2908,7 +2829,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -2993,7 +2913,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -3126,7 +3045,6 @@ mod tests { search_request.clone(), MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await?; @@ -3309,7 +3227,6 @@ mod tests { search_request.clone(), MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await?; @@ -3434,7 +3351,6 @@ mod tests { search_request, mock_metastore_client.clone(), &cluster_client, - None, ) .await .unwrap(); @@ -3454,7 +3370,6 @@ mod tests { search_request, mock_metastore_client, &cluster_client, - None, ) .await .unwrap_err(); @@ -3580,7 +3495,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -3721,7 +3635,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -3803,7 +3716,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -3871,7 +3783,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -3962,7 +3873,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -4045,7 +3955,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -4095,7 +4004,6 @@ mod tests { }, metastore.clone(), &cluster_client, - None, ) .await .is_err() @@ -4112,7 +4020,6 @@ mod tests { }, metastore, &cluster_client, - None, ) .await .is_err() @@ -4178,7 +4085,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await; assert!(search_response.is_err()); @@ -4229,7 +4135,6 @@ mod tests { search_request, metastore.clone(), &cluster_client, - None, ) .await; assert!(search_response.is_err()); @@ -4250,7 +4155,6 @@ mod tests { search_request, metastore, &cluster_client, - None, ) .await; assert!(search_response.is_err()); @@ -4769,7 +4673,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -5036,7 +4939,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -5219,7 +5121,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -5341,7 +5242,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap(); @@ -5395,7 +5295,6 @@ mod tests { search_request, MetastoreServiceClient::from_mock(mock_metastore), &cluster_client, - None, ) .await .unwrap_err(); diff --git a/quickwit/quickwit-search/src/scroll_context.rs b/quickwit/quickwit-search/src/scroll_context.rs index 949e23a65a2..a4a31a856b5 100644 --- a/quickwit/quickwit-search/src/scroll_context.rs +++ b/quickwit/quickwit-search/src/scroll_context.rs @@ -113,7 +113,6 @@ impl ScrollContext { &self.search_request, &self.split_metadatas[..], cluster_client, - None, // Lambda not supported for scroll queries ) .await?; self.cached_partial_hits_start_offset = start_offset; diff --git a/quickwit/quickwit-search/src/search_permit_provider.rs b/quickwit/quickwit-search/src/search_permit_provider.rs index 2ceb8ec5d59..13b18fa0e63 100644 --- a/quickwit/quickwit-search/src/search_permit_provider.rs +++ b/quickwit/quickwit-search/src/search_permit_provider.rs @@ -43,6 +43,12 @@ pub enum SearchPermitMessage { permit_sender: oneshot::Sender>, permit_sizes: Vec, }, + RequestWithOffload { + permit_sender: oneshot::Sender, + permit_sizes: Vec, + /// Maximum number of additional pending requests before offloading. + offload_threshold: usize, + }, UpdateMemory { memory_delta: i64, }, @@ -53,6 +59,16 @@ pub enum SearchPermitMessage { }, } +/// Result of a permit request that supports offloading some splits to Lambda. +pub struct PermitPartition { + /// Permits for splits to process locally. + /// Each entry is (split_index, permit_future) where split_index + /// refers to the position in the original request. + pub local: Vec<(usize, SearchPermitFuture)>, + /// Indices of splits that should be offloaded to Lambda. + pub offloaded: Vec, +} + /// Makes very pessimistic estimate of the memory allocation required for a split search /// /// This is refined later on when more data is available about the split. @@ -123,6 +139,32 @@ impl SearchPermitProvider { .await .expect("Receiver lives longer than sender") } + + /// Returns permits for local splits and a list of split indices to offload. + /// + /// The actor checks the current pending queue depth. If adding all splits + /// would exceed `offload_threshold` pending requests, only enough splits + /// to fill up to the threshold are processed locally; the rest are offloaded. + /// + /// If `offload_threshold` is 0, all splits go local (no offloading). + pub async fn get_permits_with_offload( + &self, + splits: impl IntoIterator, + offload_threshold: usize, + ) -> PermitPartition { + let (permit_sender, permit_receiver) = oneshot::channel(); + let permit_sizes = splits.into_iter().map(|size| size.as_u64()).collect(); + self.message_sender + .send(SearchPermitMessage::RequestWithOffload { + permit_sender, + permit_sizes, + offload_threshold, + }) + .expect("Receiver lives longer than sender"); + permit_receiver + .await + .expect("Receiver lives longer than sender") + } } struct SearchPermitActor { @@ -162,11 +204,33 @@ impl SearchPermitActor { permits.push(SearchPermitFuture(rx)); } self.assign_available_permits(); - // The receiver could be dropped in the (unlikely) situation - // where the future requesting these permits is cancelled before - // this message is processed. let _ = permit_sender.send(permits); } + SearchPermitMessage::RequestWithOffload { + permit_sizes, + permit_sender, + offload_threshold, + } => { + let current_pending = self.permits_requests.len(); + // How many new splits can we accept locally before hitting the threshold. + let local_capacity = offload_threshold.saturating_sub(current_pending); + let num_local = permit_sizes.len().min(local_capacity); + + let mut local = Vec::with_capacity(num_local); + let mut offloaded = Vec::with_capacity(permit_sizes.len() - num_local); + + for (idx, permit_size) in permit_sizes.into_iter().enumerate() { + if idx < num_local { + let (tx, rx) = oneshot::channel(); + self.permits_requests.push_back((tx, permit_size)); + local.push((idx, SearchPermitFuture(rx))); + } else { + offloaded.push(idx); + } + } + self.assign_available_permits(); + let _ = permit_sender.send(PermitPartition { local, offloaded }); + } SearchPermitMessage::UpdateMemory { memory_delta } => { if self.total_memory_allocated as i64 + memory_delta < 0 { panic!("More memory released than allocated, should never happen.") diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index b074557f730..ec75a212f14 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -53,7 +53,6 @@ pub struct SearchServiceImpl { cluster_client: ClusterClient, searcher_context: Arc, local_kv_store: MiniKV, - lambda_invoker: Option>, } /// Trait representing a search service. @@ -143,7 +142,6 @@ impl SearchServiceImpl { storage_resolver: StorageResolver, cluster_client: ClusterClient, searcher_context: Arc, - lambda_invoker: Option>, ) -> Self { SearchServiceImpl { metastore, @@ -151,7 +149,6 @@ impl SearchServiceImpl { cluster_client, searcher_context, local_kv_store: MiniKV::default(), - lambda_invoker, } } } @@ -171,7 +168,6 @@ impl SearchService for SearchServiceImpl { search_request, self.metastore.clone(), &self.cluster_client, - self.lambda_invoker.clone(), ) .await?; Ok(search_result) @@ -421,6 +417,8 @@ pub struct SearcherContext { pub list_fields_cache: ListFieldsCache, /// The aggregation limits are passed to limit the memory usage. pub aggregation_limit: AggregationLimitsGuard, + /// Optional Lambda invoker for offloading leaf search to serverless functions. + pub lambda_invoker: Option>, } impl std::fmt::Debug for SearcherContext { @@ -436,11 +434,15 @@ impl SearcherContext { #[cfg(test)] pub fn for_test() -> SearcherContext { let searcher_config = SearcherConfig::default(); - SearcherContext::new(searcher_config, None) + SearcherContext::new(searcher_config, None, None) } /// Creates a new searcher context, given a searcher config, and an optional `SplitCache`. - pub fn new(searcher_config: SearcherConfig, split_cache_opt: Option>) -> Self { + pub fn new( + searcher_config: SearcherConfig, + split_cache_opt: Option>, + lambda_invoker: Option>, + ) -> Self { let global_split_footer_cache = MemorySizedCache::from_config( &searcher_config.split_footer_cache, &quickwit_storage::STORAGE_METRICS.split_footer_cache, @@ -469,6 +471,7 @@ impl SearcherContext { list_fields_cache, split_cache_opt, aggregation_limit, + lambda_invoker, } } diff --git a/quickwit/quickwit-search/src/tests.rs b/quickwit/quickwit-search/src/tests.rs index 37acf663a1e..e93ae0d398f 100644 --- a/quickwit/quickwit-search/src/tests.rs +++ b/quickwit/quickwit-search/src/tests.rs @@ -1029,7 +1029,7 @@ async fn test_search_util(test_sandbox: &TestSandbox, query: &str) -> Vec { ..Default::default() }); let searcher_context: Arc = - Arc::new(SearcherContext::new(SearcherConfig::default(), None)); + Arc::new(SearcherContext::new(SearcherConfig::default(), None, None)); let agg_limits = searcher_context.get_aggregation_limits(); @@ -1669,7 +1669,7 @@ async fn test_single_node_list_terms() -> anyhow::Result<()> { .into_iter() .map(|split| extract_split_and_footer_offsets(&split.split_metadata)) .collect(); - let searcher_context = Arc::new(SearcherContext::new(SearcherConfig::default(), None)); + let searcher_context = Arc::new(SearcherContext::new(SearcherConfig::default(), None, None)); { let request = ListTermsRequest { diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs index 21488ddc08f..db14c37700a 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/rest_handler.rs @@ -412,7 +412,6 @@ fn build_request_for_es_api( search_after, count_hits, ignore_missing_indexes, - execution_mode: 1, // Default to Lambda (RemoteFunction) }, has_doc_id_field, )) diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index cfe69b5cc96..400e1e4261a 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -630,9 +630,19 @@ pub async fn serve_quickwit( None }; + // Initialize Lambda invoker if enabled + let lambda_invoker_opt = if let Some(lambda_config) = &node_config.searcher_config.lambda { + info!("initializing AWS Lambda invoker for leaf search"); + let invoker = quickwit_lambda_client::try_get_or_deploy_invoker(lambda_config).await?; + Some(invoker) + } else { + None + }; + let searcher_context = Arc::new(SearcherContext::new( node_config.searcher_config.clone(), split_cache_opt, + lambda_invoker_opt, )); let (search_job_placer, search_service) = setup_searcher( @@ -1018,21 +1028,11 @@ async fn setup_searcher( let searcher_pool = SearcherPool::default(); let search_job_placer = SearchJobPlacer::new(searcher_pool.clone()); - // Initialize Lambda invoker if enabled - let lambda_invoker_opt = if let Some(lambda_config) = &node_config.searcher_config.lambda { - info!("initializing AWS Lambda invoker for leaf search"); - let invoker = quickwit_lambda_client::try_get_or_deploy_invoker(lambda_config).await?; - Some(invoker) - } else { - None - }; - let search_service = start_searcher_service( metastore, storage_resolver, search_job_placer.clone(), searcher_context, - lambda_invoker_opt, ) .await?; let search_service_clone = search_service.clone(); @@ -1565,7 +1565,7 @@ mod tests { async fn test_setup_searcher() { let node_config = NodeConfig::for_test(); let searcher_context = - Arc::new(SearcherContext::new(SearcherConfig::default(), None)); + Arc::new(SearcherContext::new(SearcherConfig::default(), None, None)); let metastore = metastore_for_test(); let (change_stream, change_stream_tx) = ClusterChangeStream::new_unbounded(); let storage_resolver = StorageResolver::unconfigured(); diff --git a/quickwit/quickwit-serve/src/search_api/rest_handler.rs b/quickwit/quickwit-serve/src/search_api/rest_handler.rs index f1ac6196998..cfdf46c61ef 100644 --- a/quickwit/quickwit-serve/src/search_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/search_api/rest_handler.rs @@ -265,7 +265,6 @@ pub fn search_request_from_api_request( search_after: None, count_hits: search_request.count_all.into(), ignore_missing_indexes: false, - execution_mode: 1, // Default to Lambda (RemoteFunction) }; Ok(search_request) }