From c335f759e055210761f923ca7fe013f44b62d020 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Thu, 12 Feb 2026 13:33:56 +0100 Subject: [PATCH 1/6] Offload leaf search work to AWS Lambda functions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The goal is to handle traffic spikes gracefully without provisioning additional searcher nodes: when the local search queue is saturated, overflow splits are transparently routed to Lambda for processing. The offloading decision happens **on the leaf side**, inside the `SearchPermitProvider`. The permit provider already manages a bounded queue of pending split search tasks (gated by memory budget and download slots). When a leaf search request arrives, the provider checks the current queue depth against a configurable `offload_threshold`. If granting permits for all requested splits would exceed this threshold, only enough splits to fill up to the threshold are processed locally — the rest are marked for offloading. The offloaded splits are batched (up to `max_splits_per_invocation` splits per batch, balanced by document count) and sent to Lambda in parallel. Each Lambda invocation runs the same leaf search code path and **returns per-split results individually**. This is important: the per-split responses are fed back into the `IncrementalCollector` and populate the **partial result cache**, so subsequent queries hitting the same splits benefit from cached results regardless of whether the split was searched locally or on Lambda. Depending on the configuration, the Lambda function code can be **deployed automatically** at startup. The `quickwit-lambda-client` crate embeds a compressed Lambda binary at compile time. When `auto_deploy` is configured, Quickwit will: 1. Check if a published Lambda version matching the current binary already exists (identified by a description tag `quickwit:{version}-{hash}`) 2. Create or update the function and publish a new version if needed 3. Garbage-collect old versions (keeping the current one + 5 most recent) This ensures the Lambda function always matches the running Quickwit version without any external deployment tooling. Manual deployment is also supported for users who prefer to manage Lambda functions through Terraform or other IaC tools. Lambda offloading is opt-in. Add a `lambda` section under `searcher` in the node configuration: ```yaml searcher: lambda: offload_threshold: 100 # queue depth before offloading kicks in (0 = always offload) 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 ``` - **`quickwit-lambda-client`**: Handles Lambda invocation (with metrics) and auto-deployment logic. Embeds the Lambda binary at build time. - **`quickwit-lambda-server`**: The Lambda function handler itself — receives a `LeafSearchRequest`, runs `multi_index_leaf_search`, and returns per-split `LeafSearchResponse`s. - **`quickwit-search`**: New `LambdaLeafSearchInvoker` trait; `SearchPermitProvider` gains `get_permits_with_offload` to split work between local and offloaded; `leaf.rs` orchestrates local and Lambda tasks in parallel. - **`quickwit-config`**: New `LambdaConfig` and `LambdaDeployConfig` structs under `SearcherConfig`. - **`quickwit-serve`**: Initializes the Lambda invoker at startup when configured. - **`quickwit-proto`**: New `LeafSearchResponses` wrapper message for batched per-split responses. build fix --- .../cargo-build-macos-binary/action.yml | 2 +- .github/actions/cross-build-binary/action.yml | 2 +- .github/workflows/publish_lambda.yaml | 78 ++ LICENSE-3rdparty.csv | 8 + docs/configuration/lambda-config.md | 217 ++++ quickwit/Cargo.lock | 202 +++- quickwit/Cargo.toml | 11 + quickwit/quickwit-aws/src/lib.rs | 2 +- quickwit/quickwit-config/Cargo.toml | 1 + .../resources/tests/node_config/quickwit.json | 10 + .../resources/tests/node_config/quickwit.toml | 10 + .../resources/tests/node_config/quickwit.yaml | 8 + quickwit/quickwit-config/src/lib.rs | 4 +- .../quickwit-config/src/node_config/mod.rs | 90 +- .../src/node_config/serialize.rs | 13 +- quickwit/quickwit-lambda-client/Cargo.toml | 45 + quickwit/quickwit-lambda-client/README.md | 51 + quickwit/quickwit-lambda-client/build.rs | 107 ++ quickwit/quickwit-lambda-client/src/deploy.rs | 972 ++++++++++++++++++ .../quickwit-lambda-client/src/invoker.rs | 185 ++++ quickwit/quickwit-lambda-client/src/lib.rs | 37 + .../quickwit-lambda-client/src/metrics.rs | 74 ++ quickwit/quickwit-lambda-server/Cargo.toml | 53 + .../src/bin/leaf_search.rs | 50 + .../quickwit-lambda-server/src/context.rs | 69 ++ quickwit/quickwit-lambda-server/src/error.rs | 69 ++ .../quickwit-lambda-server/src/handler.rs | 131 +++ quickwit/quickwit-lambda-server/src/lib.rs | 26 + quickwit/quickwit-lambda/README.md | 4 - .../protos/quickwit/search.proto | 6 + .../src/codegen/quickwit/quickwit.search.rs | 8 + quickwit/quickwit-proto/src/error.rs | 2 +- quickwit/quickwit-search/src/invoker.rs | 35 + quickwit/quickwit-search/src/leaf.rs | 711 +++++++++++-- quickwit/quickwit-search/src/leaf_cache.rs | 1 - quickwit/quickwit-search/src/lib.rs | 8 +- quickwit/quickwit-search/src/list_terms.rs | 22 +- quickwit/quickwit-search/src/root.rs | 4 +- .../src/search_permit_provider.rs | 195 +++- quickwit/quickwit-search/src/service.rs | 15 +- quickwit/quickwit-search/src/tests.rs | 7 +- quickwit/quickwit-serve/Cargo.toml | 1 + quickwit/quickwit-serve/src/lib.rs | 18 +- .../src/cache/memory_sized_cache.rs | 10 + 44 files changed, 3383 insertions(+), 191 deletions(-) create mode 100644 .github/workflows/publish_lambda.yaml create mode 100644 docs/configuration/lambda-config.md create mode 100644 quickwit/quickwit-lambda-client/Cargo.toml create mode 100644 quickwit/quickwit-lambda-client/README.md create mode 100644 quickwit/quickwit-lambda-client/build.rs create mode 100644 quickwit/quickwit-lambda-client/src/deploy.rs create mode 100644 quickwit/quickwit-lambda-client/src/invoker.rs create mode 100644 quickwit/quickwit-lambda-client/src/lib.rs create mode 100644 quickwit/quickwit-lambda-client/src/metrics.rs create mode 100644 quickwit/quickwit-lambda-server/Cargo.toml create mode 100644 quickwit/quickwit-lambda-server/src/bin/leaf_search.rs create mode 100644 quickwit/quickwit-lambda-server/src/context.rs create mode 100644 quickwit/quickwit-lambda-server/src/error.rs create mode 100644 quickwit/quickwit-lambda-server/src/handler.rs create mode 100644 quickwit/quickwit-lambda-server/src/lib.rs delete mode 100644 quickwit/quickwit-lambda/README.md create mode 100644 quickwit/quickwit-search/src/invoker.rs diff --git a/.github/actions/cargo-build-macos-binary/action.yml b/.github/actions/cargo-build-macos-binary/action.yml index 53e9df53dc3..4db9ce46198 100644 --- a/.github/actions/cargo-build-macos-binary/action.yml +++ b/.github/actions/cargo-build-macos-binary/action.yml @@ -60,7 +60,7 @@ runs: path: ./${{ env.ASSET_FULL_NAME }}.tar.gz retention-days: 3 - name: Deploy archive to GitHub release - uses: quickwit-inc/upload-to-github-release@v1 + uses: quickwit-inc/upload-to-github-release@9b2c40fba23bf8dea05b7d2eece24cbc95d4a190 env: GITHUB_TOKEN: ${{ inputs.token }} with: diff --git a/.github/actions/cross-build-binary/action.yml b/.github/actions/cross-build-binary/action.yml index e5b855154ea..0f23b17aa22 100644 --- a/.github/actions/cross-build-binary/action.yml +++ b/.github/actions/cross-build-binary/action.yml @@ -56,7 +56,7 @@ runs: path: ./${{ env.ASSET_FULL_NAME }}.tar.gz retention-days: 3 - name: Upload archive - uses: quickwit-inc/upload-to-github-release@v1 + uses: quickwit-inc/upload-to-github-release@9b2c40fba23bf8dea05b7d2eece24cbc95d4a190 env: GITHUB_TOKEN: ${{ inputs.token }} with: diff --git a/.github/workflows/publish_lambda.yaml b/.github/workflows/publish_lambda.yaml new file mode 100644 index 00000000000..80b5386f0bf --- /dev/null +++ b/.github/workflows/publish_lambda.yaml @@ -0,0 +1,78 @@ +# 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-client/README.md +name: Release Lambda binary + +on: + push: + tags: + - 'lambda-*' + 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.ref_type }}" = "tag" ]; then + # Extract version from tag (e.g., lambda-v0.8.0 -> v0.8.0) + echo "ASSET_VERSION=${GITHUB_REF_NAME#lambda-}" >> $GITHUB_ENV + elif [ -n "${{ github.event.inputs.version }}" ] && [ "${{ github.event.inputs.version }}" != "dev" ]; then + echo "ASSET_VERSION=${{ github.event.inputs.version }}" >> $GITHUB_ENV + else + echo "ASSET_VERSION=dev-$(git rev-parse --short HEAD)" >> $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-server --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@9b2c40fba23bf8dea05b7d2eece24cbc95d4a190 + 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 594ac812cf7..b5c3eaf3c34 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 " @@ -222,6 +225,8 @@ itoa,https://github.com/dtolnay/itoa,MIT OR Apache-2.0,David Tolnay 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 @@ -397,6 +402,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 @@ -497,9 +503,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/docs/configuration/lambda-config.md b/docs/configuration/lambda-config.md new file mode 100644 index 00000000000..4803d1a2418 --- /dev/null +++ b/docs/configuration/lambda-config.md @@ -0,0 +1,217 @@ +--- +title: Lambda configuration +sidebar_position: 6 +--- + +Quickwit supports offloading leaf search operations to AWS Lambda for horizontal scaling. When the local search queue becomes saturated, overflow splits are automatically sent to Lambda functions for processing. + +:::note +Lambda offloading is currently only supported on AWS. +::: + +## How it works + +Lambda offloading is **only active when a `lambda` configuration section is present** under `searcher` in your node configuration. When configured: + +1. Quickwit monitors the local search queue depth +2. When pending searches exceed the `offload_threshold`, new splits are sent to Lambda instead of being queued locally +3. Lambda returns per-split search results that are cached and merged with local results + +This allows Quickwit to handle traffic spikes without provisioning additional searcher nodes. + +## Startup validation + +When a `lambda` configuration is defined, Quickwit performs a **dry run invocation** at startup to verify that: +- The Lambda function exists +- The function version matches the embedded binary +- The invoker has permission to call the function + +If this validation fails, **Quickwit will fail to start**. This ensures that Lambda offloading works correctly before the node begins serving traffic. + +## Configuration + +Add a `lambda` section under `searcher` in your node configuration: + +```yaml +searcher: + lambda: + offload_threshold: 100 + auto_deploy: + execution_role_arn: arn:aws:iam::123456789012:role/quickwit-lambda-role + memory_size: 5 GiB + invocation_timeout_secs: 15 +``` + +### Lambda configuration options + +| Property | Description | Default value | +| --- | --- | --- | +| `function_name` | Name of the AWS Lambda function to invoke. | `quickwit-lambda-search` | +| `max_splits_per_invocation` | Maximum number of splits to send in a single Lambda invocation. Must be at least 1. | `10` | +| `offload_threshold` | Number of pending local searches before offloading to Lambda. A value of `0` offloads everything to Lambda. | `100` | +| `auto_deploy` | Auto-deployment configuration. If set, Quickwit automatically deploys or updates the Lambda function at startup. | (none) | + +### Auto-deploy configuration options + +| Property | Description | Default value | +| --- | --- | --- | +| `execution_role_arn` | **Required.** IAM role ARN for the Lambda function's execution role. | | +| `memory_size` | Memory allocated to the Lambda function. More memory provides more CPU. | `5 GiB` | +| `invocation_timeout_secs` | Timeout for Lambda invocations in seconds. | `15` | + +## Deployment options + +### Automatic deployment (recommended) + +With `auto_deploy` configured, Quickwit automatically: +1. Creates the Lambda function if it doesn't exist +2. Updates the function code if the embedded binary has changed +3. Publishes a new version with a unique identifier +4. Garbage collects old versions (keeps current + 5 most recent) + +This is the recommended approach as it ensures the Lambda function always matches the Quickwit binary version. + +### Manual deployment + +You can deploy the Lambda function manually without `auto_deploy`: +1. Download the Lambda zip from [GitHub releases](https://github.com/quickwit-oss/quickwit/releases) +2. Create or update the Lambda function using AWS CLI, Terraform, or the AWS Console +3. Publish a version with description format `quickwit:{version}-{sha1}` (e.g., `quickwit:0_8_0-fa752891`) + +The description must match the format Quickwit expects, or it won't find the function version. + +## IAM permissions + +### Permissions for the Quickwit node + +The IAM role or user running Quickwit needs the following permissions to invoke Lambda: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Action": [ + "lambda:InvokeFunction" + ], + "Resource": "arn:aws:lambda:*:*:function:quickwit-lambda-search:*" + } + ] +} +``` + +If using `auto_deploy`, additional permissions are required for deployment: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Action": [ + "lambda:CreateFunction", + "lambda:GetFunction", + "lambda:UpdateFunctionCode", + "lambda:PublishVersion", + "lambda:ListVersionsByFunction", + "lambda:DeleteFunction" + ], + "Resource": "arn:aws:lambda:*:*:function:quickwit-lambda-search" + }, + { + "Effect": "Allow", + "Action": "iam:PassRole", + "Resource": "arn:aws:iam::*:role/quickwit-lambda-role", + "Condition": { + "StringEquals": { + "iam:PassedToService": "lambda.amazonaws.com" + } + } + } + ] +} +``` + +### Lambda execution role + +The Lambda function requires an execution role with S3 read access to your index data. CloudWatch logging permissions are not required. + +Example policy: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::your-index-bucket/*" + } + ] +} +``` + +The execution role must also have a trust policy allowing Lambda to assume it: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Principal": { + "Service": "lambda.amazonaws.com" + }, + "Action": "sts:AssumeRole" + } + ] +} +``` + +## Versioning + +Quickwit uses content-based versioning for Lambda: +- A SHA1 hash of the Lambda binary is computed at build time +- This hash is embedded in the Lambda function description as `quickwit:{version}-{sha1_short}` +- When Quickwit starts, it searches for a version matching this description +- Different Quickwit builds with the same Lambda binary share the same Lambda version +- Updating the Lambda binary automatically triggers a new deployment + +## Example configuration + + +Minimal configuration (with auto-deployment): + +```yaml +searcher: + lambda: + auto_deploy: + execution_role_arn: arn:aws:iam::123456789012:role/quickwit-lambda-role +``` + + +Full configuration (auto-deployment): + +```yaml +searcher: + lambda: + function_name: quickwit-lambda-search + max_splits_per_invocation: 10 + offload_threshold: 10 + auto_deploy: + execution_role_arn: arn:aws:iam::123456789012:role/quickwit-lambda-role + memory_size: 5 GiB + invocation_timeout_secs: 15 +``` + +Aggressive offloading (send everything to Lambda): + +```yaml +searcher: + lambda: + function_name: quickwit-lambda-search + offload_threshold: 0 + auto_deploy: + execution_role_arn: arn:aws:iam::123456789012:role/quickwit-lambda-role +``` diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 21f1a789bec..53261a6b54b 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" @@ -685,9 +708,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.7" +version = "1.2.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ee19095c7c4dda59f1697d028ce704c24b2d33c6718790c7f1d5a3015b4107c" +checksum = "3cba48474f1d6807384d06fec085b909f5807e16653c5af5c45dfe89539f0b70" dependencies = [ "futures-util", "pin-project-lite", @@ -771,9 +794,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.5" +version = "1.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59e62db736db19c488966c8d787f52e6270be565727236fd5579eaa301e7bc4a" +checksum = "0709f0083aa19b704132684bc26d3c868e06bd428ccc4373b0b55c3e8748a58b" dependencies = [ "aws-smithy-async", "aws-smithy-protocol-test", @@ -823,6 +846,18 @@ dependencies = [ "aws-smithy-types", ] +[[package]] +name = "aws-smithy-mocks" +version = "0.2.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "943deb750d91da9f788024482ff1f3393cefb017d0dc30fc5cecb11e1be15770" +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" @@ -834,9 +869,9 @@ dependencies = [ [[package]] name = "aws-smithy-protocol-test" -version = "0.63.7" +version = "0.63.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01317a9e3c5c06f1af35001ef0c873c1e34e458c20b2ee1eee0fb431e6dbb010" +checksum = "b59f9305f7863a70f4a0c048fa6d81fb9dd9373a751358791faaad8881c1377f" dependencies = [ "assert-json-diff", "aws-smithy-runtime-api", @@ -888,9 +923,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.9.3" +version = "1.11.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab0d43d899f9e508300e587bf582ba54c27a452dd0a9ea294690669138ae14a2" +checksum = "8c55e0837e9b8526f49e0b9bfa9ee18ddee70e853f5bc09c5d11ebceddcb0fec" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -905,9 +940,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.5" +version = "1.4.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "905cb13a9895626d49cf2ced759b062d913834c7482c38e49557eac4e6193f01" +checksum = "576b0d6991c9c32bc14fc340582ef148311f924d41815f641a308b5d11e8e7cd" dependencies = [ "base64-simd", "bytes", @@ -4328,6 +4363,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" @@ -7009,6 +7093,56 @@ dependencies = [ "utoipa", ] +[[package]] +name = "quickwit-lambda-client" +version = "0.8.0" +dependencies = [ + "anyhow", + "async-trait", + "aws-config", + "aws-sdk-lambda", + "aws-smithy-mocks", + "base64 0.22.1", + "bytesize", + "md5", + "once_cell", + "prost 0.14.1", + "quickwit-common", + "quickwit-config", + "quickwit-lambda-server", + "quickwit-proto", + "quickwit-search", + "quickwit-storage", + "serde_json", + "tokio", + "tracing", + "ureq", + "zip", +] + +[[package]] +name = "quickwit-lambda-server" +version = "0.8.0" +dependencies = [ + "anyhow", + "base64 0.22.1", + "bytesize", + "lambda_runtime", + "openssl", + "prost 0.14.1", + "quickwit-common", + "quickwit-config", + "quickwit-proto", + "quickwit-search", + "quickwit-storage", + "serde", + "serde_json", + "thiserror 2.0.17", + "tokio", + "tracing", + "tracing-subscriber", +] + [[package]] name = "quickwit-macros" version = "0.8.0" @@ -7260,6 +7394,7 @@ dependencies = [ "quickwit-ingest", "quickwit-jaeger", "quickwit-janitor", + "quickwit-lambda-client", "quickwit-metastore", "quickwit-opentelemetry", "quickwit-proto", @@ -10288,6 +10423,35 @@ version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" +[[package]] +name = "ureq" +version = "3.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdc97a28575b85cfedf2a7e7d3cc64b3e11bd8ac766666318003abbacc7a21fc" +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" @@ -10317,6 +10481,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" @@ -11380,6 +11550,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 c2e8ec62dc3..78655a74b0d 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -19,6 +19,8 @@ members = [ "quickwit-integration-tests", "quickwit-jaeger", "quickwit-janitor", + "quickwit-lambda-client", + "quickwit-lambda-server", "quickwit-macros", "quickwit-metastore", @@ -56,6 +58,8 @@ default-members = [ "quickwit-integration-tests", "quickwit-jaeger", "quickwit-janitor", + "quickwit-lambda-client", + "quickwit-lambda-server", "quickwit-macros", "quickwit-metastore", "quickwit-opentelemetry", @@ -142,6 +146,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" lru = "0.16" @@ -283,6 +288,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 @@ -297,6 +303,7 @@ vrl = { version = "0.29", default-features = false, features = [ ] } warp = { version = "0.4", features = ["server", "test"] } wiremock = "0.6" +zip = { version = "0.6", default-features = false, features = ["deflate"] } zstd = { version = "0.13", default-features = false } aws-config = "1.8" @@ -304,8 +311,10 @@ 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-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 = [ @@ -344,6 +353,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-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-aws/src/lib.rs b/quickwit/quickwit-aws/src/lib.rs index 31b63d121e2..2dec49509bc 100644 --- a/quickwit/quickwit-aws/src/lib.rs +++ b/quickwit/quickwit-aws/src/lib.rs @@ -42,5 +42,5 @@ pub async fn get_aws_config() -> &'static aws_config::SdkConfig { /// Returns the AWS behavior version. pub fn aws_behavior_version() -> BehaviorVersion { - BehaviorVersion::v2025_08_07() + BehaviorVersion::v2026_01_12() } 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..7269b37ae22 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.json +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.json @@ -70,6 +70,16 @@ "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, + "offload_threshold": 30, + "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..ea715dcffe0 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.toml @@ -62,6 +62,16 @@ 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 +offload_threshold = 30 + +[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..face0852972 100644 --- a/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml +++ b/quickwit/quickwit-config/resources/tests/node_config/quickwit.yaml @@ -64,6 +64,14 @@ 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 + offload_threshold: 30 + 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/lib.rs b/quickwit/quickwit-config/src/lib.rs index df108b844c0..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, 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 e8c347eb4a5..de2dc3bb4a0 100644 --- a/quickwit/quickwit-config/src/node_config/mod.rs +++ b/quickwit/quickwit-config/src/node_config/mod.rs @@ -300,6 +300,85 @@ 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. + /// 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)] +pub struct LambdaConfig { + /// AWS Lambda function name. + #[serde(default = "LambdaConfig::default_function_name")] + pub function_name: String, + /// Maximum number of splits per Lambda invocation. + #[serde(default = "LambdaConfig::default_max_splits_per_invocation")] + pub max_splits_per_invocation: NonZeroUsize, + /// Maximum number of splits to process locally before offloading to Lambda. + /// 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 offloads everything to Lambda. + #[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. + #[serde(default)] + pub auto_deploy: Option, +} + +impl LambdaConfig { + #[cfg(feature = "testsuite")] + pub fn for_test() -> Self { + 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, + } + } + + fn default_function_name() -> String { + "quickwit-lambda-search".to_string() + } + fn default_max_splits_per_invocation() -> NonZeroUsize { + NonZeroUsize::new(10).unwrap() + } + fn default_offload_threshold() -> usize { + 100 + } +} + +/// Configuration for automatic Lambda function deployment. +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(deny_unknown_fields)] +pub struct LambdaDeployConfig { + /// IAM execution role ARN for the Lambda function. + /// The role only requires GetObject permission to the targeted S3 bucket. + pub execution_role_arn: String, + /// 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 LambdaDeployConfig { + fn default_memory_size() -> ByteSize { + // Empirically this implies between 4 and 6 vCPUs. + ByteSize::gib(5) + } + fn default_invocation_timeout_secs() -> u64 { + 15 + } } #[derive(Clone, Debug, PartialEq, Eq, Hash, Serialize, Deserialize)] @@ -316,6 +395,14 @@ pub struct CacheConfig { } impl CacheConfig { + pub fn no_cache() -> Self { + CacheConfig { + capacity: None, + policy: None, + virtual_caches: Vec::new(), + } + } + pub fn default_with_capacity(capacity: ByteSize) -> Self { CacheConfig { capacity: Some(capacity), @@ -376,7 +463,7 @@ pub enum CachePolicy { } impl std::fmt::Display for CachePolicy { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { CachePolicy::Lru => f.write_str("lru"), CachePolicy::S3Fifo => f.write_str("s3-fifo"), @@ -435,6 +522,7 @@ impl Default for SearcherConfig { storage_timeout_policy: None, warmup_memory_budget: ByteSize::gb(100), warmup_single_split_initial_allocation: ByteSize::gb(1), + lambda: None, } } } diff --git a/quickwit/quickwit-config/src/node_config/serialize.rs b/quickwit/quickwit-config/src/node_config/serialize.rs index 7d3b5f2c64b..e9b9b745e8b 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,17 @@ 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: NonZeroUsize::new(10).unwrap(), + offload_threshold: 30, + 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-client/Cargo.toml b/quickwit/quickwit-lambda-client/Cargo.toml new file mode 100644 index 00000000000..c96f39e46de --- /dev/null +++ b/quickwit/quickwit-lambda-client/Cargo.toml @@ -0,0 +1,45 @@ +[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] +anyhow = { workspace = true } +async-trait = { workspace = true } +aws-config = { workspace = true } +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 } + +[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] +md5 = { workspace = true } +ureq = { workspace = true } +zip = { workspace = true, default-features = false, features = ["deflate"] } diff --git a/quickwit/quickwit-lambda-client/README.md b/quickwit/quickwit-lambda-client/README.md new file mode 100644 index 00000000000..94d4fbf965b --- /dev/null +++ b/quickwit/quickwit-lambda-client/README.md @@ -0,0 +1,51 @@ +# Quickwit Lambda + +Quickwit supports offloading leaf search to AWS Lambda for horizontal scaling. +The Lambda function is built separately and embedded into Quickwit's binary, +allowing Quickwit to auto-deploy the function at startup. + +## Architecture + +- **quickwit-lambda-server**: The Lambda function binary that executes leaf searches +- **quickwit-lambda-client**: The client that invokes Lambda and embeds the Lambda zip for auto-deployment + +## Release Process + +### 1. Tag the release + +Push a tag with the `lambda-` prefix: + +```bash +git tag lambda-v0.8.0 +git push origin lambda-v0.8.0 +``` + +This triggers the `publish_lambda.yaml` GitHub Action which: +- Cross-compiles the Lambda binary for ARM64 +- Creates a zip file named `quickwit-aws-lambda-v0.8.0-aarch64.zip` +- Uploads it as a **draft** GitHub release + +### 2. Publish the release + +Go to GitHub releases and manually publish the draft release to make the +artifact URL publicly accessible. + +### 3. Update the embedded Lambda URL + +Update `LAMBDA_ZIP_URL` in `quickwit-lambda-client/build.rs` to point to the +new release: + +```rust +const LAMBDA_ZIP_URL: &str = "https://github.com/quickwit-oss/quickwit/releases/download/lambda-v0.8.0/quickwit-aws-lambda-v0.8.0-aarch64.zip"; +``` + +### 4. Versioning + +The Lambda client uses content-based versioning: +- An MD5 hash of the Lambda zip is computed at build time +- This hash is embedded in the Lambda function description as `quickwit:{version}-{hash_short}` +- When Quickwit starts, it checks if a matching version exists before deploying + +This ensures that: +- Different Quickwit builds with the same Lambda binary share the same Lambda version +- Updating the Lambda binary automatically triggers a new deployment diff --git a/quickwit/quickwit-lambda-client/build.rs b/quickwit/quickwit-lambda-client/build.rs new file mode 100644 index 00000000000..193e7a054fa --- /dev/null +++ b/quickwit/quickwit-lambda-client/build.rs @@ -0,0 +1,107 @@ +// 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-client. +//! +//! 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. + +use std::env; +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. +const LAMBDA_ZIP_URL: &str = "https://github.com/quickwit-oss/quickwit/releases/download/lambda-ff6fdfa5/quickwit-aws-lambda--aarch64.zip"; + +/// AWS Lambda direct upload limit is 50MB. +/// Larger artifacts must be uploaded via S3. +const MAX_LAMBDA_ZIP_SIZE: usize = 50 * 1024 * 1024; + +fn main() { + println!("cargo:rerun-if-changed=build.rs"); + + let out_dir = PathBuf::from(env::var("OUT_DIR").expect("OUT_DIR not set")); + let zip_path = out_dir.join("lambda_bootstrap.zip"); + let url_marker_path = out_dir.join("lambda_bootstrap.url"); + + // Check if we already have the zip from the same URL + let needs_download = if zip_path.try_exists().unwrap() && url_marker_path.try_exists().unwrap() + { + let cached_url = std::fs::read_to_string(&url_marker_path).unwrap_or_default(); + cached_url.trim() != LAMBDA_ZIP_URL + } else { + true + }; + + let lambda_zip_payload: Vec = if needs_download { + println!( + "cargo:warning=Downloading Lambda zip from: {}", + LAMBDA_ZIP_URL + ); + let data: Vec = + download_lambda_zip(LAMBDA_ZIP_URL).expect("failed to download lambda zip"); + std::fs::write(&zip_path, &data).expect("Failed to write zip file"); + std::fs::write(&url_marker_path, LAMBDA_ZIP_URL).expect("Failed to write URL marker"); + println!( + "cargo:warning=Downloaded Lambda zip to {:?} ({} bytes)", + zip_path, + data.len() + ); + data + } else { + println!("Using cached Lambda zip from {:?}", zip_path); + std::fs::read(&zip_path).expect("Failed to read cached zip file") + }; + + // Compute MD5 hash of the zip and export as environment variable. + // This is used to create a unique qualifier for Lambda versioning. + let digest = md5::compute(&lambda_zip_payload); + let hash_short = &format!("{:x}", digest)[..8]; // First 8 hex chars + println!("cargo:rustc-env=LAMBDA_BINARY_HASH={}", hash_short); + println!("Lambda binary hash (short): {}", hash_short); +} + +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 + ); + } + validate_zip(&data)?; + Ok(data) +} + +fn validate_zip(data: &[u8]) -> Result<(), Box> { + let cursor = std::io::Cursor::new(data); + let mut archive = zip::ZipArchive::new(cursor)?; + // Verify we can read all entries (checks CRC for each file) + for i in 0..archive.len() { + let mut file = archive.by_index(i)?; + std::io::copy(&mut file, &mut std::io::sink())?; + } + Ok(()) +} diff --git a/quickwit/quickwit-lambda-client/src/deploy.rs b/quickwit/quickwit-lambda-client/src/deploy.rs new file mode 100644 index 00000000000..e7890bfc059 --- /dev/null +++ b/quickwit/quickwit-lambda-client/src/deploy.rs @@ -0,0 +1,972 @@ +// 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; + +/// 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}-{hash_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_HASH") + ) + }) + .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 hash, +/// 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.clone(), 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() => { + info!( + 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() + && description == target_description + && let Some(ver) = version.version() + && 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 { + // This looks overly complicated but this is not AI slop. + // The AWS API forces us to go through a bunch of hoops to update our function + // in a safe manner. + + // Fast path: if the function does not exist, we can create and publish the function atomically. + if let Some(version) = try_create_function(client, function_name, deploy_config).await? { + return Ok(version); + } + + // Function already exists — we need to update the code. + // This will create or update a version called "$LATEST" (that's the actual string) + // + // We cannot directly publish here, because updating the function code does not allow + // use to pass a different description. + let code_sha256 = update_function_code(client, function_name).await?; + + // We can now publish that new uploaded version. + // We pass the code_sha256 guard to make sure a race condition does not cause + // us to publish a different version. + // + // Publishing will create an actual version (a number as a string) and return it. + 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() + .div_ceil(1024u64 * 1024u64) 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) + && let Ok(version_num) = version_str.parse::() + { + quickwit_lambda_versions.push((version_num, version_str.to_string())); + } + } + + marker = response.next_marker().map(ToString::to_string); + if marker.is_none() { + break; + } + } + + // Sort by version number ascending (oldest 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 +} + +#[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(&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-lambda-client/src/invoker.rs b/quickwit/quickwit-lambda-client/src/invoker.rs new file mode 100644 index 00000000000..9c4204be346 --- /dev/null +++ b/quickwit/quickwit-lambda-client/src/invoker.rs @@ -0,0 +1,185 @@ +// 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; + +use anyhow::Context as _; +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_lambda_server::{LeafSearchRequestPayload, LeafSearchResponsePayload}; +use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse, LeafSearchResponses}; +use quickwit_search::{LambdaLeafSearchInvoker, SearchError}; +use tracing::{debug, info, instrument}; + +use crate::metrics::LAMBDA_METRICS; + +/// Create a Lambda invoker for a specific version. +/// +/// 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: String, + version: String, +) -> anyhow::Result> { + let aws_config = aws_config::load_defaults(aws_config::BehaviorVersion::latest()).await; + let client = LambdaClient::new(&aws_config); + let invoker = AwsLambdaInvoker { + client, + function_name, + version, + }; + invoker.validate().await?; + Ok(Arc::new(invoker)) +} + +/// AWS Lambda implementation of RemoteFunctionInvoker. +struct AwsLambdaInvoker { + client: LambdaClient, + function_name: String, + /// The version number to invoke (e.g., "7", "12"). + version: String, +} + +impl AwsLambdaInvoker { + /// Validate that the Lambda function version exists and is invocable. + /// Uses DryRun invocation type - validates without executing. + async fn validate(&self) -> anyhow::Result<()> { + 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.with_context(|| { + format!( + "failed to validate Lambda function '{}:{}'", + self.function_name, self.version + ) + })?; + + info!("the lambda invoker dry run was successful"); + Ok(()) + } +} + +#[async_trait] +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, SearchError> { + 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, SearchError> { + // Serialize request to protobuf bytes, then base64 encode + let request_bytes = request.encode_to_vec(); + let payload = LeafSearchRequestPayload { + payload: BASE64_STANDARD.encode(&request_bytes), + }; + + 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 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 + .map_err(|e| SearchError::Internal(format!("Lambda invocation error: {}", e)))?; + + // 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(SearchError::Internal(format!( + "lambda function error: {}: {}", + error, error_payload + ))); + } + + // Deserialize response + let response_payload = response + .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)))?; + + let response_bytes = BASE64_STANDARD + .decode(&lambda_response.payload) + .map_err(|e| SearchError::Internal(format!("base64 decode error: {}", e)))?; + + let leaf_responses = LeafSearchResponses::decode(&response_bytes[..]) + .map_err(|e| SearchError::Internal(format!("protobuf decode error: {}", e)))?; + + debug!( + num_responses = leaf_responses.responses.len(), + "lambda invocation completed" + ); + + Ok(leaf_responses.responses) + } +} diff --git a/quickwit/quickwit-lambda-client/src/lib.rs b/quickwit/quickwit-lambda-client/src/lib.rs new file mode 100644 index 00000000000..f1bee07cb5b --- /dev/null +++ b/quickwit/quickwit-lambda-client/src/lib.rs @@ -0,0 +1,37 @@ +// 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 client for Quickwit leaf search operations. +//! +//! This crate provides: +//! - An AWS Lambda implementation of the `LambdaLeafSearchInvoker` trait used by `quickwit-search` +//! - Auto-deployment functionality for Lambda functions +//! +//! # Usage +//! +//! Use `try_get_or_deploy_invoker` to get an invoker that will automatically deploy +//! the Lambda function if needed: +//! +//! ```ignore +//! let invoker = try_get_or_deploy_invoker(&function_name, &deploy_config).await?; +//! ``` + +mod deploy; +mod invoker; +mod metrics; + +pub use deploy::try_get_or_deploy_invoker; +pub use metrics::LAMBDA_METRICS; +// Re-export payload types from server crate for convenience +pub use quickwit_lambda_server::{LeafSearchRequestPayload, LeafSearchResponsePayload}; diff --git a/quickwit/quickwit-lambda-client/src/metrics.rs b/quickwit/quickwit-lambda-client/src/metrics.rs new file mode 100644 index 00000000000..4325c9b3a95 --- /dev/null +++ b/quickwit/quickwit-lambda-client/src/metrics.rs @@ -0,0 +1,74 @@ +// 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 100ms to 73s seconds +fn duration_buckets() -> Vec { + exponential_buckets(0.100, 3f64.sqrt(), 13).unwrap() +} + +/// From 1KB to 16MB +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, +} + +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(), + ), + } + } +} + +pub static LAMBDA_METRICS: Lazy = Lazy::new(LambdaMetrics::default); diff --git a/quickwit/quickwit-lambda-server/Cargo.toml b/quickwit/quickwit-lambda-server/Cargo.toml new file mode 100644 index 00000000000..1bfebfcc655 --- /dev/null +++ b/quickwit/quickwit-lambda-server/Cargo.toml @@ -0,0 +1,53 @@ +[package] +name = "quickwit-lambda-server" +description = "AWS Lambda handler 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 + +[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 } +base64 = { 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"] } + +openssl = { workspace = true, optional = true } + +quickwit-common = { workspace = true } +quickwit-config = { workspace = true } +quickwit-proto = { workspace = true } +quickwit-search = { workspace = true } +quickwit-storage = { workspace = true } + +[[bin]] +name = "quickwit-aws-lambda-leaf-search" +path = "src/bin/leaf_search.rs" + +[features] +default = [] +testsuite = [] + +# 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", +] diff --git a/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs b/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs new file mode 100644 index 00000000000..6a4dacab392 --- /dev/null +++ b/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs @@ -0,0 +1,50 @@ +// 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 std::sync::Arc; + +use lambda_runtime::{Error, LambdaEvent, service_fn}; +use quickwit_lambda_server::{LambdaSearcherContext, LeafSearchRequestPayload, handle_leaf_search}; +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(); + + // 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| { + 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-server/src/context.rs b/quickwit/quickwit-lambda-server/src/context.rs new file mode 100644 index 00000000000..879572e5f8d --- /dev/null +++ b/quickwit/quickwit-lambda-server/src/context.rs @@ -0,0 +1,69 @@ +// 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; + +use anyhow::Context as _; +use bytesize::ByteSize; +use quickwit_config::{CacheConfig, SearcherConfig}; +use quickwit_search::SearcherContext; +use quickwit_storage::StorageResolver; +use tracing::info; + +/// Lambda-specific searcher context that holds resources for search execution. +pub struct LambdaSearcherContext { + pub searcher_context: Arc, + pub storage_resolver: StorageResolver, +} + +impl LambdaSearcherContext { + /// Create a new Lambda searcher context from environment variables. + pub fn try_from_env() -> anyhow::Result { + info!("initializing lambda searcher context"); + + let searcher_config = try_searcher_config_from_env()?; + let searcher_context = Arc::new(SearcherContext::new(searcher_config, None, None)); + let storage_resolver = StorageResolver::configured(&Default::default()); + + Ok(Self { + searcher_context, + storage_resolver, + }) + } +} + +/// Create a Lambda-optimized searcher config based on the `AWS_LAMBDA_FUNCTION_MEMORY_SIZE` +/// environment variable. +fn try_searcher_config_from_env() -> anyhow::Result { + let lambda_memory_mib: u64 = quickwit_common::get_from_env_opt( + "AWS_LAMBDA_FUNCTION_MEMORY_SIZE", + /* sensitive */ false, + ) + .context("could not get aws lambda function memory size from ENV")?; + let lambda_memory = ByteSize::mib(lambda_memory_mib); + anyhow::ensure!( + lambda_memory >= ByteSize::gib(1u64), + "lambda memory must be at least 1GB" + ); + let warmup_memory_budget = ByteSize::b(lambda_memory.as_u64() - ByteSize::mib(500).as_u64()); + + let mut searcher_config = SearcherConfig::default(); + searcher_config.max_num_concurrent_split_searches = 20; + searcher_config.warmup_memory_budget = warmup_memory_budget; + searcher_config.fast_field_cache = CacheConfig::no_cache(); + searcher_config.split_footer_cache = CacheConfig::no_cache(); + searcher_config.predicate_cache = CacheConfig::no_cache(); + searcher_config.partial_request_cache = CacheConfig::no_cache(); + Ok(searcher_config) +} diff --git a/quickwit/quickwit-lambda-server/src/error.rs b/quickwit/quickwit-lambda-server/src/error.rs new file mode 100644 index 00000000000..7f60f33b9b8 --- /dev/null +++ b/quickwit/quickwit-lambda-server/src/error.rs @@ -0,0 +1,69 @@ +// 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 quickwit_search::SearchError; +use thiserror::Error; + +/// Result type for Lambda operations. +pub type LambdaResult = Result; + +/// Errors that can occur during Lambda handler operations. +#[derive(Debug, Error)] +pub enum LambdaError { + /// Error serializing/deserializing protobuf. + #[error("serialization error: {0}")] + Serialization(String), + /// Error from the search operation. + #[error("search error: {0}")] + Search(#[from] SearchError), + /// Internal error. + #[error("internal error: {0}")] + Internal(String), + /// Task was cancelled. + #[error("cancelled")] + Cancelled, +} + +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-server/src/handler.rs b/quickwit/quickwit-lambda-server/src/handler.rs new file mode 100644 index 00000000000..15387975a3c --- /dev/null +++ b/quickwit/quickwit-lambda-server/src/handler.rs @@ -0,0 +1,131 @@ +// 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, LeafSearchResponses, SplitIdAndFooterOffsets, +}; +use quickwit_search::leaf::multi_index_leaf_search; +use serde::{Deserialize, Serialize}; +use tracing::{error, info, instrument, warn}; + +use crate::context::LambdaSearcherContext; +use crate::error::{LambdaError, LambdaResult}; + +/// Payload for leaf search Lambda invocation. +#[derive(Debug, Serialize, Deserialize)] +pub struct LeafSearchRequestPayload { + /// Base64-encoded serialized LeafSearchRequest protobuf. + pub payload: String, +} + +/// Response from leaf search Lambda invocation. +#[derive(Debug, Serialize, Deserialize)] +pub struct LeafSearchResponsePayload { + /// 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: LeafSearchRequestPayload, + 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 all_splits: Vec<(usize, SplitIdAndFooterOffsets)> = + leaf_search_request + .leaf_requests + .iter() + .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(); + + let num_splits = all_splits.len(); + info!(num_splits, "processing leaf search request (per-split)"); + + // Process each split in parallel. The SearchPermitProvider inside + // SearcherContext gates concurrency based on memory budget. + let mut split_search_futures: Vec> = + Vec::with_capacity(all_splits.len()); + for (leaf_req_idx, split) in all_splits { + 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(); + split_search_futures.push(tokio::task::spawn(multi_index_leaf_search( + searcher_context, + single_split_request, + storage_resolver, + ))); + } + + // Collect results, preserving split order. + let mut responses: Vec = Vec::with_capacity(num_splits); + for split_search_fut in split_search_futures { + match split_search_fut.await { + Ok(Ok(response)) => responses.push(response), + Ok(Err(e)) => { + return Err(LambdaError::Internal(format!("leaf search failed: {e}"))); + } + Err(join_error) if join_error.is_cancelled() => { + warn!("search task was cancelled"); + return Err(LambdaError::Cancelled); + } + Err(join_error) => { + error!(error = %join_error, "search task panicked"); + return Err(LambdaError::Internal(format!( + "search task panicked: {join_error}" + ))); + } + } + } + + info!( + num_responses = responses.len(), + "leaf search completed (per-split)" + ); + + // 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-lambda-server/src/lib.rs b/quickwit/quickwit-lambda-server/src/lib.rs new file mode 100644 index 00000000000..78f882d995f --- /dev/null +++ b/quickwit/quickwit-lambda-server/src/lib.rs @@ -0,0 +1,26 @@ +// 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 context; +mod error; +mod handler; + +pub use context::LambdaSearcherContext; +pub use error::{LambdaError, LambdaResult}; +pub use handler::{LeafSearchRequestPayload, LeafSearchResponsePayload, handle_leaf_search}; diff --git a/quickwit/quickwit-lambda/README.md b/quickwit/quickwit-lambda/README.md deleted file mode 100644 index 88fa9c8748a..00000000000 --- a/quickwit/quickwit-lambda/README.md +++ /dev/null @@ -1,4 +0,0 @@ -# Deprecation - -This package was removed in Q3 2025. The maintenance burden was high and the -feature was unused. \ No newline at end of file diff --git a/quickwit/quickwit-proto/protos/quickwit/search.proto b/quickwit/quickwit-proto/protos/quickwit/search.proto index 12443855239..d6a79ddace5 100644 --- a/quickwit/quickwit-proto/protos/quickwit/search.proto +++ b/quickwit/quickwit-proto/protos/quickwit/search.proto @@ -497,6 +497,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 191110365f5..997b93b6869 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs @@ -460,6 +460,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 { 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/invoker.rs b/quickwit/quickwit-search/src/invoker.rs new file mode 100644 index 00000000000..f160d88019f --- /dev/null +++ b/quickwit/quickwit-search/src/invoker.rs @@ -0,0 +1,35 @@ +// 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 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, SearchError>; +} diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 4caf2587909..ccee5d6e27a 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; +use std::num::NonZeroUsize; use std::ops::Bound; use std::path::PathBuf; use std::str::FromStr; @@ -23,6 +24,7 @@ use anyhow::Context; use bytesize::ByteSize; use futures::future::try_join_all; use quickwit_common::pretty::PrettySample; +use quickwit_common::uri::Uri; use quickwit_directories::{CachingDirectory, HotDirectory, StorageDirectory}; use quickwit_doc_mapper::{Automaton, DocMapper, FastFieldWarmupInfo, TermRange, WarmupInfo}; use quickwit_proto::search::{ @@ -37,8 +39,8 @@ use quickwit_storage::{ BundleStorage, ByteRangeCache, MemorySizedCache, OwnedBytes, SplitCache, Storage, StorageResolver, TimeoutAndRetryStorage, wrap_storage_with_cache, }; +use tantivy::aggregation::AggContextParams; use tantivy::aggregation::agg_req::{AggregationVariants, Aggregations}; -use tantivy::aggregation::{AggContextParams, AggregationLimitsGuard}; use tantivy::collector::Collector; use tantivy::directory::FileSlice; use tantivy::fastfield::FastFieldReaders; @@ -48,12 +50,64 @@ use tokio::task::{JoinError, JoinSet}; use tracing::*; use crate::collector::{IncrementalCollector, make_collector_for_split, make_merge_collector}; +use crate::leaf_cache::LeafSearchCache; use crate::metrics::SplitSearchOutcomeCounters; use crate::root::is_metadata_count_request_with_ast; -use crate::search_permit_provider::{SearchPermit, compute_initial_memory_allocation}; +use crate::search_permit_provider::{ + SearchPermit, SearchPermitFuture, compute_initial_memory_allocation, +}; use crate::service::{SearcherContext, deserialize_doc_mapper}; use crate::{QuickwitAggregations, SearchError}; +/// Distributes items across batches using a greedy LPT (Longest Processing Time) +/// algorithm to balance total weight across batches. +/// +/// Items are sorted by weight descending, then each item is assigned to the +/// batch with the smallest current total weight. This produces a good +/// approximation of balanced batches. +fn greedy_batch_split( + items: Vec, + weight_fn: impl Fn(&T) -> u64, + max_items_per_batch: NonZeroUsize, +) -> Vec> { + if items.is_empty() { + return Vec::new(); + } + + let num_items = items.len(); + let max_items_per_batch: usize = max_items_per_batch.get(); + let num_batches = num_items.div_ceil(max_items_per_batch); + + // Compute weights, then sort descending by weight + let mut weighted_items: Vec<(u64, T)> = Vec::with_capacity(num_items); + for item in items { + let weight = weight_fn(&item); + weighted_items.push((weight, item)); + } + weighted_items.sort_unstable_by_key(|(weight, _)| std::cmp::Reverse(*weight)); + + // Invariant: batch_weights[i] is the sum of weights in batches[i] + let mut batches: Vec> = std::iter::repeat_with(Vec::new).take(num_batches).collect(); + let mut batch_weights: Vec = vec![0; num_batches]; + + // Greedily assign each item to the lightest batch. + // Ties are broken by count to help balance item counts when weights are equal. + for (weight, item) in weighted_items { + let lightest_batch_idx = batch_weights + .iter() + .zip(batches.iter()) + .enumerate() + .filter(|(_, (_, batch))| batch.len() < max_items_per_batch) + .min_by_key(|(_, (batch_weight, batch))| (**batch_weight, batch.len())) + .map(|(idx, _)| idx) + .unwrap(); + batch_weights[lightest_batch_idx] += weight; + batches[lightest_batch_idx].push(item); + } + + batches +} + async fn get_split_footer_from_cache_or_fetch( index_storage: Arc, split_and_footer_offsets: &SplitIdAndFooterOffsets, @@ -435,43 +489,26 @@ fn compute_index_size(hot_directory: &HotDirectory) -> ByteSize { /// Apply a leaf search on a single split. #[allow(clippy::too_many_arguments)] async fn leaf_search_single_split( - mut search_request: SearchRequest, + search_request: SearchRequest, ctx: Arc, storage: Arc, split: SplitIdAndFooterOffsets, - aggregations_limits: AggregationLimitsGuard, search_permit: &mut SearchPermit, ) -> crate::Result> { let mut leaf_search_state_guard = SplitSearchStateGuard::new(ctx.split_outcome_counters.clone()); - rewrite_request( - &mut search_request, - &split, - ctx.doc_mapper.timestamp_field_name(), - ); - let query_ast: QueryAst = serde_json::from_str(search_request.query_ast.as_str()) .map_err(|err| SearchError::InvalidQuery(err.to_string()))?; // CanSplitDoBetter or rewrite_request may have changed the request to be a count only request // This may be the case for AllQuery with a sort by date and time filter, where the current // split can't have better results. - // if is_metadata_count_request_with_ast(&query_ast, &search_request) { leaf_search_state_guard.set_state(SplitSearchState::PrunedBeforeWarmup); return Ok(Some(get_leaf_resp_from_count(split.num_docs))); } - if let Some(cached_answer) = ctx - .searcher_context - .leaf_search_cache - .get(split.clone(), search_request.clone()) - { - leaf_search_state_guard.set_state(SplitSearchState::CacheHit); - return Ok(Some(cached_answer)); - } - let split_id = split.split_id.to_string(); let byte_range_cache = ByteRangeCache::with_infinite_capacity(&quickwit_storage::STORAGE_METRICS.shortlived_cache); @@ -489,14 +526,14 @@ async fn leaf_search_single_split( search_permit.update_memory_usage(index_size); } - let reader = index + let searcher = index .reader_builder() .reload_policy(ReloadPolicy::Manual) - .try_into()?; - let searcher = reader.searcher(); + .try_into()? + .searcher(); let agg_context_params = AggContextParams { - limits: aggregations_limits, + limits: ctx.searcher_context.get_aggregation_limits(), tokenizers: ctx.doc_mapper.tokenizer_manager().tantivy_manager().clone(), }; let mut collector = @@ -595,16 +632,14 @@ async fn leaf_search_single_split( })??; // Let's cache this result in the partial result cache. - if let Some((leaf_search_req, leaf_search_resp)) = search_request_and_result { - ctx.searcher_context.leaf_search_cache.put( - split, - leaf_search_req, - leaf_search_resp.clone(), - ); - Ok(Some(leaf_search_resp)) - } else { - Ok(None) - } + let Some((leaf_search_req, leaf_search_resp)) = search_request_and_result else { + return Ok(None); + }; + // We save our result in the cache. + ctx.searcher_context + .leaf_search_cache + .put(split, leaf_search_req, leaf_search_resp.clone()); + Ok(Some(leaf_search_resp)) } /// Rewrite a request removing parts which incur additional download or computation with no @@ -685,7 +720,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 { @@ -1061,12 +1097,12 @@ impl CanSplitDoBetter { /// Returns the search_requests with their split. fn optimize( &self, - request: Arc, + request: &SearchRequest, mut splits: Vec, ) -> Result, SearchError> { self.optimize_split_order(&mut splits); - if !is_simple_all_query(&request) { + if !is_simple_all_query(request) { // no optimization opportunity here. return Ok(splits .into_iter() @@ -1211,7 +1247,7 @@ impl CanSplitDoBetter { pub async fn multi_index_leaf_search( searcher_context: Arc, leaf_search_request: LeafSearchRequest, - storage_resolver: &StorageResolver, + storage_resolver: StorageResolver, ) -> Result { let search_request: Arc = leaf_search_request .search_request @@ -1223,8 +1259,7 @@ pub async fn multi_index_leaf_search( .iter() .map(|doc_mapper| deserialize_doc_mapper(doc_mapper)) .collect::>()?; - // Creates a collector which merges responses into one - let aggregation_limits = searcher_context.get_aggregation_limits(); + // TODO: to avoid lockstep, we should pull up the future creation over the list of split ids // and have the semaphore on this level. // This will lower resource consumption due to less in-flight futures and avoid contention. @@ -1255,11 +1290,11 @@ pub async fn multi_index_leaf_search( })? .clone(); + let storage_resolver = storage_resolver.clone(); + let searcher_context = searcher_context.clone(); + let search_request = search_request.clone(); + leaf_request_futures.spawn({ - let storage_resolver = storage_resolver.clone(); - let searcher_context = searcher_context.clone(); - let search_request = search_request.clone(); - let aggregation_limits = aggregation_limits.clone(); async move { let storage = storage_resolver.resolve(&index_uri).await?; single_doc_mapping_leaf_search( @@ -1268,7 +1303,6 @@ pub async fn multi_index_leaf_search( storage, leaf_search_request_ref.split_offsets, doc_mapper, - aggregation_limits, ) .in_current_span() .await @@ -1276,8 +1310,11 @@ pub async fn multi_index_leaf_search( }); } - let merge_collector = make_merge_collector(&search_request, aggregation_limits)?; + // Creates a collector which merges responses into one + let merge_collector = + make_merge_collector(&search_request, searcher_context.get_aggregation_limits())?; let mut incremental_merge_collector = IncrementalCollector::new(merge_collector); + while let Some(leaf_response_join_result) = leaf_request_futures.join_next().await { // abort the search on join errors let leaf_response_result = leaf_response_join_result?; @@ -1352,9 +1389,169 @@ fn disable_search_request_hits(search_request: &mut SearchRequest) { } /// Searches multiple splits for a specific index and a single doc mapping -/// +/// Offloads splits to Lambda invocations, distributing them accross batches +/// balanced by document count. Each batch is invoked independently; a failure +/// in one batch does not affect others. +async fn run_offloaded_search_tasks( + searcher_context: &SearcherContext, + request: &SearchRequest, + doc_mapper: &DocMapper, + index_uri: Uri, + splits: Vec, + incremental_merge_collector: &Mutex, +) -> Result<(), SearchError> { + if splits.is_empty() { + return Ok(()); + } + + info!(num_offloaded_splits = splits.len(), "offloading to lambda"); + + let lambda_invoker = searcher_context.lambda_invoker.as_ref().expect( + "did not receive enough permit futures despite not having any lambda invoker to offload to", + ); + let lambda_config = searcher_context.searcher_config.lambda.as_ref().unwrap(); + + let doc_mapper_str = serde_json::to_string(doc_mapper) + .map_err(|err| SearchError::Internal(format!("failed to serialize doc mapper: {err}")))?; + + let batches: Vec> = greedy_batch_split( + splits, + |split| split.num_docs, + lambda_config.max_splits_per_invocation, + ); + + 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_joinset = JoinSet::new(); + for batch in batches { + let batch_split_ids: Vec = + batch.iter().map(|split| split.split_id.clone()).collect(); + let leaf_request = LeafSearchRequest { + search_request: Some(search_request_for_leaf.clone()), + doc_mappers: vec![doc_mapper_str.clone()], + index_uris: vec![index_uri.as_str().to_string()], //< careful here. Calling to_string() directly would return a redacted uri. + leaf_requests: vec![quickwit_proto::search::LeafRequestRef { + index_uri_ord: 0, + doc_mapper_ord: 0, + split_offsets: batch, + }], + }; + let invoker = lambda_invoker.clone(); + lambda_tasks_joinset.spawn(async move { + ( + batch_split_ids, + invoker.invoke_leaf_search(leaf_request).await, + ) + }); + } + + while let Some(join_res) = lambda_tasks_joinset.join_next().await { + let Ok((batch_split_ids, result)) = join_res else { + error!("lambda join error"); + return Err(SearchError::Internal("lambda join error".to_string())); + }; + let mut locked = incremental_merge_collector.lock().unwrap(); + match result { + Ok(per_split_responses) => { + for response in per_split_responses { + if let Err(err) = locked.add_result(response) { + error!(error = %err, "failed to add lambda result to collector"); + } + } + } + Err(err) => { + error!( + error = %err, + num_splits = batch_split_ids.len(), + "lambda invocation failed for batch" + ); + for split_id in batch_split_ids { + locked.add_failed_split(SplitSearchError { + split_id, + error: format!("lambda invocation error: {err}"), + retryable_error: true, + }); + } + } + } + } + + Ok(()) +} + +struct LocalSearchTask { + split: SplitIdAndFooterOffsets, + search_request: SearchRequest, + search_permit_future: SearchPermitFuture, +} + +struct ScheduleSearchTaskResult { + // The search permit futures associated to each local_search_task are + // guaranteed to resolve in order. + local_search_tasks: Vec, + offloaded_search_tasks: Vec, +} + +/// Schedule search tasks, either: +/// - locally +/// - remotely on lambdas, if lambda are configured, and the number of tasks scheduled exceed the +/// offload threshold. +async fn schedule_search_tasks( + mut splits: Vec<(SplitIdAndFooterOffsets, SearchRequest)>, + searcher_context: &SearcherContext, +) -> ScheduleSearchTaskResult { + let permit_sizes: Vec = splits + .iter() + .map(|(split, _)| { + compute_initial_memory_allocation( + split, + searcher_context + .searcher_config + .warmup_single_split_initial_allocation, + ) + }) + .collect(); + + let offload_threshold: usize = if searcher_context.lambda_invoker.is_some() + && let Some(lambda_config) = &searcher_context.searcher_config.lambda + { + lambda_config.offload_threshold + } else { + usize::MAX + }; + + let search_permit_futures = searcher_context + .search_permit_provider + .get_permits_with_offload(permit_sizes, offload_threshold) + .await; + + let splits_to_run_on_lambda: Vec = splits + .drain(search_permit_futures.len()..) + .map(|(split, _req)| split) + .collect(); + + let splits_to_run_locally: Vec = splits + .into_iter() + .zip(search_permit_futures) + .map( + |((split, search_request), search_permit_future)| LocalSearchTask { + split, + search_request, + search_permit_future, + }, + ) + .collect(); + + ScheduleSearchTaskResult { + local_search_tasks: splits_to_run_locally, + offloaded_search_tasks: splits_to_run_on_lambda, + } +} + /// 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( @@ -1363,55 +1560,113 @@ pub async fn single_doc_mapping_leaf_search( index_storage: Arc, splits: Vec, doc_mapper: Arc, - aggregations_limits: AggregationLimitsGuard, ) -> Result { let num_docs: u64 = splits.iter().map(|split| split.num_docs).sum(); let num_splits = splits.len(); info!(num_docs, num_splits, split_offsets = ?PrettySample::new(&splits, 5)); - let split_filter = CanSplitDoBetter::from_request(&request, doc_mapper.timestamp_field_name()); - let split_with_req = split_filter.optimize(request.clone(), splits)?; + // We simplify the request as much as possible. + let split_filter: CanSplitDoBetter = + CanSplitDoBetter::from_request(&request, doc_mapper.timestamp_field_name()); + let mut split_with_req: Vec<(SplitIdAndFooterOffsets, SearchRequest)> = + split_filter.optimize(&request, splits)?; + for (split, search_request) in &mut split_with_req { + rewrite_request(search_request, split, doc_mapper.timestamp_field_name()); + } + let split_filter_arc: Arc> = Arc::new(RwLock::new(split_filter)); - let split_filter = Arc::new(RwLock::new(split_filter)); + let merge_collector = + make_merge_collector(&request, searcher_context.get_aggregation_limits())?; + let mut incremental_merge_collector = IncrementalCollector::new(merge_collector); - let merge_collector = make_merge_collector(&request, aggregations_limits.clone())?; - let incremental_merge_collector = IncrementalCollector::new(merge_collector); - let incremental_merge_collector = Arc::new(Mutex::new(incremental_merge_collector)); + let split_outcome_counters = Arc::new(SplitSearchOutcomeCounters::new_unregistered()); - // 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; + // Sort out the splits that are already in the partial result cache. + let uncached_splits: Vec<(SplitIdAndFooterOffsets, SearchRequest)> = + process_partial_result_cache( + &searcher_context.leaf_search_cache, + split_with_req, + split_outcome_counters.clone(), + &mut incremental_merge_collector, + )?; + let incremental_merge_collector_arc: Arc> = + Arc::new(Mutex::new(incremental_merge_collector)); + + // Determine which uncached splits to process locally vs offload. + let ScheduleSearchTaskResult { + local_search_tasks, + offloaded_search_tasks, + } = schedule_search_tasks(uncached_splits, &searcher_context).await; + + // Offload splits to Lambda. + let run_offloaded_search_tasks_fut = run_offloaded_search_tasks( + &searcher_context, + &request, + &doc_mapper, + index_storage.uri().clone(), + offloaded_search_tasks, + &incremental_merge_collector_arc, + ); + // Spawn local split search tasks. let leaf_search_context = Arc::new(LeafSearchContext { searcher_context: searcher_context.clone(), - split_outcome_counters: Arc::new(SplitSearchOutcomeCounters::new_unregistered()), - incremental_merge_collector: incremental_merge_collector.clone(), + split_outcome_counters, + incremental_merge_collector: incremental_merge_collector_arc.clone(), doc_mapper: doc_mapper.clone(), - split_filter: split_filter.clone(), + split_filter: split_filter_arc.clone(), }); + let run_local_search_tasks_fut = run_local_search_tasks( + local_search_tasks, + index_storage, + split_filter_arc, + leaf_search_context, + ); + + let (offloaded_res, _) = + tokio::join!(run_offloaded_search_tasks_fut, run_local_search_tasks_fut); + offloaded_res?; - 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()) + // we can't use unwrap_or_clone because mutexes aren't Clone + let incremental_merge_collector = match Arc::try_unwrap(incremental_merge_collector_arc) { + Ok(filter_merger) => filter_merger.into_inner().unwrap(), + Err(filter_merger) => filter_merger.lock().unwrap().clone(), + }; + + let leaf_search_response_result: tantivy::Result = + crate::search_thread_pool() + .run_cpu_intensive(|| incremental_merge_collector.finalize()) + .instrument(info_span!("incremental_merge_intermediate")) + .await + .context("failed to merge split search responses: thread panicked")?; + + Ok(leaf_search_response_result?) +} + +async fn run_local_search_tasks( + local_search_tasks: Vec, + index_storage: Arc, + split_filter_arc: Arc>, + leaf_search_context: Arc, +) { + let mut split_search_joinset = JoinSet::new(); + let mut task_id_to_split_id_map = HashMap::with_capacity(local_search_tasks.len()); + + for LocalSearchTask { + split, + search_request, + search_permit_future, + } in local_search_tasks { - let leaf_split_search_permit = permit_fut + let leaf_split_search_permit = search_permit_future .instrument(info_span!("waiting_for_leaf_search_split_semaphore")) .await; + // We run simplify search request again: as we push split into the merge collector, + // we may have discovered that we won't find any better candidates for top hits in this + // split, in which case we can remove top hits collection. let Some(simplified_search_request) = - simplify_search_request(search_request, &split, &split_filter) + simplify_search_request(search_request, &split, &split_filter_arc) else { let mut leaf_search_state_guard = SplitSearchStateGuard::new(leaf_search_context.split_outcome_counters.clone()); @@ -1419,29 +1674,25 @@ pub async fn single_doc_mapping_leaf_search( continue; }; let split_id = split.split_id.clone(); - let handle = split_search_futures.spawn( + let handle = split_search_joinset.spawn( leaf_search_single_split_wrapper( simplified_search_request, leaf_search_context.clone(), index_storage.clone(), - split, + split.clone(), leaf_split_search_permit, - aggregations_limits.clone(), ) .in_current_span(), ); 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 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 + while let Some(leaf_search_join_result) = split_search_joinset.join_next().await { 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(); @@ -1454,30 +1705,44 @@ pub async fn single_doc_mapping_leaf_search( } } - info!(split_outcome_counters=%leaf_search_context.split_outcome_counters, "leaf split search finished"); - - // we can't use unwrap_or_clone because mutexes aren't Clone - let mut 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(), - }; - + let mut incremental_merge_collector_lock = leaf_search_context + .incremental_merge_collector + .lock() + .unwrap(); for (split_id, split_search_join_error) in split_search_join_errors { - incremental_merge_collector.add_failed_split(SplitSearchError { + incremental_merge_collector_lock.add_failed_split(SplitSearchError { split_id, error: SearchError::from(split_search_join_error).to_string(), retryable_error: true, }); } - let leaf_search_response_reresult: Result, _> = - crate::search_thread_pool() - .run_cpu_intensive(|| incremental_merge_collector.finalize()) - .instrument(info_span!("incremental_merge_intermediate")) - .await - .context("failed to merge split search responses"); + info!(split_outcome_counters=%leaf_search_context.split_outcome_counters, "leaf split search finished"); +} - Ok(leaf_search_response_reresult??) +/// We identify the splits that are in the cache and append them to the incremental merge collector. +/// The (split, request) that are yet to be processed are returned. +fn process_partial_result_cache( + leaf_search_cache: &LeafSearchCache, + split_with_req: Vec<(SplitIdAndFooterOffsets, SearchRequest)>, + split_outcome_counters: Arc, + incremental_merge_collector: &mut IncrementalCollector, +) -> Result, SearchError> { + let mut uncached_splits: Vec<(SplitIdAndFooterOffsets, SearchRequest)> = + Vec::with_capacity(split_with_req.len()); + for (split, search_request) in split_with_req { + if let Some(cached_response) = leaf_search_cache + // TODO remove the clone here. + .get(split.clone(), search_request.clone()) + { + let mut split_search_guard = SplitSearchStateGuard::new(split_outcome_counters.clone()); + split_search_guard.set_state(SplitSearchState::CacheHit); + incremental_merge_collector.add_result(cached_response)?; + } else { + uncached_splits.push((split, search_request)); + } + } + Ok(uncached_splits) } #[derive(Copy, Clone)] @@ -1549,7 +1814,6 @@ async fn leaf_search_single_split_wrapper( index_storage: Arc, split: SplitIdAndFooterOffsets, mut search_permit: SearchPermit, - aggregations_limits: AggregationLimitsGuard, ) { let timer = crate::SEARCH_METRICS .leaf_search_split_duration_secs @@ -1560,7 +1824,6 @@ async fn leaf_search_single_split_wrapper( ctx.clone(), index_storage, split.clone(), - aggregations_limits, &mut search_permit, ) .await; @@ -1605,7 +1868,9 @@ async fn leaf_search_single_split_wrapper( mod tests { use std::ops::Bound; + use async_trait::async_trait; use bytes::BufMut; + use quickwit_config::{LambdaConfig, SearcherConfig}; use quickwit_directories::write_hotcache; use rand::Rng; use tantivy::TantivyDocument; @@ -1615,6 +1880,7 @@ mod tests { }; use super::*; + use crate::LambdaLeafSearchInvoker; fn bool_filter(ast: impl Into) -> QueryAst { BoolQuery { @@ -2144,4 +2410,241 @@ mod tests { assert!(directory_size_larger > directory_size_smaller + 100); assert!(larger_size > smaller_size + 100); } + + fn nz(n: usize) -> std::num::NonZeroUsize { + std::num::NonZeroUsize::new(n).unwrap() + } + + #[test] + fn test_greedy_batch_split_empty() { + let items: Vec = vec![]; + let batches = super::greedy_batch_split(items, |&x| x, nz(5)); + assert!(batches.is_empty()); + } + + #[test] + fn test_greedy_batch_split_single_batch() { + let items = vec![10u64, 20, 30]; + let batches = super::greedy_batch_split(items, |&x| x, nz(10)); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0].len(), 3); + } + + #[test] + fn test_greedy_batch_split_balances_weights() { + // 7 items with weights, max 3 per batch -> 3 batches + let items = vec![100u64, 80, 60, 50, 40, 30, 20]; + let batches = super::greedy_batch_split(items, |&x| x, nz(3)); + + assert_eq!(batches.len(), 3); + + // All items should be present + let mut all_items: Vec = batches.iter().flatten().copied().collect(); + all_items.sort_unstable(); + assert_eq!(all_items, vec![20, 30, 40, 50, 60, 80, 100]); + + // Check weights are reasonably balanced + let weights: Vec = batches.iter().map(|b| b.iter().sum()).collect(); + let max_weight = *weights.iter().max().unwrap(); + let min_weight = *weights.iter().min().unwrap(); + // With greedy LPT, the imbalance should be bounded + assert!( + max_weight <= min_weight * 2, + "weights should be reasonably balanced: {:?}", + weights + ); + } + + #[test] + fn test_greedy_batch_split_count_balance() { + // 10 items, max 3 per batch -> 4 batches + // counts should be either 2 or 3 per batch + let items: Vec = (0..10).collect(); + let batches = super::greedy_batch_split(items, |&x| x, nz(3)); + + assert_eq!(batches.len(), 4); + let counts: Vec = batches.iter().map(|b| b.len()).collect(); + for count in &counts { + assert!( + *count >= 2 && *count <= 3, + "count should be 2 or 3, got {}", + count + ); + } + assert_eq!(counts.iter().sum::(), 10); + } + + fn make_splits_with_requests( + num_splits: usize, + ) -> Vec<(SplitIdAndFooterOffsets, SearchRequest)> { + (0..num_splits) + .map(|idx| { + let split = SplitIdAndFooterOffsets { + split_id: format!("split_{idx}"), + num_docs: 100, + ..Default::default() + }; + (split, SearchRequest::default()) + }) + .collect() + } + + #[tokio::test] + async fn test_schedule_search_tasks_no_lambda_all_local() { + let searcher_context = SearcherContext::for_test(); + let splits = make_splits_with_requests(5); + let result = super::schedule_search_tasks(splits, &searcher_context).await; + assert_eq!(result.local_search_tasks.len(), 5); + assert!(result.offloaded_search_tasks.is_empty()); + for (idx, task) in result.local_search_tasks.iter().enumerate() { + assert_eq!(task.split.split_id, format!("split_{idx}")); + } + } + + struct DummyInvoker; + #[async_trait] + impl LambdaLeafSearchInvoker for DummyInvoker { + async fn invoke_leaf_search( + &self, + _req: LeafSearchRequest, + ) -> Result, SearchError> { + todo!() + } + } + + #[tokio::test] + async fn test_schedule_search_tasks_lambda_offloads_excess() { + let mut config = SearcherConfig::default(); + config.lambda = Some(LambdaConfig { + offload_threshold: 3, + ..LambdaConfig::for_test() + }); + let searcher_context = SearcherContext::new(config, None, Some(Arc::new(DummyInvoker))); + let splits = make_splits_with_requests(7); + let result = super::schedule_search_tasks(splits, &searcher_context).await; + assert_eq!(result.local_search_tasks.len(), 3); + assert_eq!(result.offloaded_search_tasks.len(), 4); + for (idx, task) in result.local_search_tasks.iter().enumerate() { + assert_eq!(task.split.split_id, format!("split_{idx}")); + } + for (idx, split) in result.offloaded_search_tasks.iter().enumerate() { + assert_eq!(split.split_id, format!("split_{}", idx + 3)); + } + } + + #[tokio::test] + async fn test_schedule_search_tasks_lambda_threshold_zero_offloads_all() { + let mut config = SearcherConfig::default(); + config.lambda = Some(LambdaConfig { + offload_threshold: 0, + ..LambdaConfig::for_test() + }); + let searcher_context = SearcherContext::new(config, None, Some(Arc::new(DummyInvoker))); + let splits = make_splits_with_requests(5); + let result = super::schedule_search_tasks(splits, &searcher_context).await; + assert!(result.local_search_tasks.is_empty()); + assert_eq!(result.offloaded_search_tasks.len(), 5); + } + + #[tokio::test] + async fn test_schedule_search_tasks_lambda_threshold_above_split_count() { + let mut config = SearcherConfig::default(); + config.lambda = Some(LambdaConfig { + offload_threshold: 100, + ..LambdaConfig::for_test() + }); + let searcher_context = SearcherContext::new(config, None, Some(Arc::new(DummyInvoker))); + let splits = make_splits_with_requests(5); + let result = super::schedule_search_tasks(splits, &searcher_context).await; + assert_eq!(result.local_search_tasks.len(), 5); + assert!(result.offloaded_search_tasks.is_empty()); + } + + #[tokio::test] + async fn test_schedule_search_tasks_empty() { + let searcher_context = SearcherContext::for_test(); + let result = super::schedule_search_tasks(Vec::new(), &searcher_context).await; + assert!(result.local_search_tasks.is_empty()); + assert!(result.offloaded_search_tasks.is_empty()); + } + + mod proptest_greedy_batch { + use std::num::NonZeroUsize; + + use proptest::prelude::*; + + proptest! { + #[test] + fn all_items_preserved( + items in prop::collection::vec(0u64..1000, 0..100), + max_per_batch in 1usize..20 + ) { + let original: Vec = items.clone(); + let max_per_batch = NonZeroUsize::new(max_per_batch).unwrap(); + let batches = super::super::greedy_batch_split(items, |&x| x, max_per_batch); + + // All items should be present exactly once + let mut result: Vec = batches.into_iter().flatten().collect(); + result.sort_unstable(); + let mut expected = original; + expected.sort_unstable(); + prop_assert_eq!(result, expected); + } + + #[test] + fn batch_count_correct( + items in prop::collection::vec(0u64..1000, 1..100), + max_per_batch in 1usize..20 + ) { + let n = items.len(); + let max_per_batch_nz = NonZeroUsize::new(max_per_batch).unwrap(); + let batches = super::super::greedy_batch_split(items, |&x| x, max_per_batch_nz); + + let expected_batches = n.div_ceil(max_per_batch); + prop_assert_eq!(batches.len(), expected_batches); + } + + #[test] + fn total_items_matches( + items in prop::collection::vec(0u64..1000, 1..100), + max_per_batch in 1usize..20 + ) { + let n = items.len(); + let max_per_batch = NonZeroUsize::new(max_per_batch).unwrap(); + let batches = super::super::greedy_batch_split(items, |&x| x, max_per_batch); + + // Total items across all batches equals input + let total: usize = batches.iter().map(|b| b.len()).sum(); + prop_assert_eq!(total, n); + } + + #[test] + fn greedy_balances_by_weight_not_count( + // Use items with significant weights to test weight balancing + items in prop::collection::vec(100u64..1000, 4..30), + max_per_batch in 2usize..10 + ) { + let max_per_batch = NonZeroUsize::new(max_per_batch).unwrap(); + let batches = super::super::greedy_batch_split(items, |&x| x, max_per_batch); + + if batches.len() >= 2 { + let weights: Vec = batches.iter().map(|b| b.iter().sum()).collect(); + let total_weight: u64 = weights.iter().sum(); + let avg_weight = total_weight / batches.len() as u64; + + // LPT guarantees max makespan <= (4/3) * optimal + // With balanced input, max should be close to average + let max_weight = *weights.iter().max().unwrap(); + + // Max weight should be at most 2x average (generous bound) + prop_assert!( + max_weight <= avg_weight * 2 + 1000, // +1000 for rounding slack + "max weight {} too far from average {}", + max_weight, + avg_weight + ); + } + } + } + } } diff --git a/quickwit/quickwit-search/src/leaf_cache.rs b/quickwit/quickwit-search/src/leaf_cache.rs index f9be57222c8..cc4d27e464e 100644 --- a/quickwit/quickwit-search/src/leaf_cache.rs +++ b/quickwit/quickwit-search/src/leaf_cache.rs @@ -70,7 +70,6 @@ impl LeafSearchCache { result: LeafSearchResponse, ) { let key = CacheKey::from_split_meta_and_request(split_info, search_request); - let encoded_result = result.encode_to_vec(); self.content.put(key, OwnedBytes::new(encoded_result)); } diff --git a/quickwit/quickwit-search/src/lib.rs b/quickwit/quickwit-search/src/lib.rs index 008556d595f..89b45a69014 100644 --- a/quickwit/quickwit-search/src/lib.rs +++ b/quickwit/quickwit-search/src/lib.rs @@ -23,7 +23,10 @@ mod collector; mod error; mod fetch_docs; mod find_trace_ids_collector; -mod leaf; + +mod invoker; +/// Leaf search operations. +pub mod leaf; mod leaf_cache; mod list_fields; mod list_fields_cache; @@ -80,6 +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::invoker::LambdaLeafSearchInvoker; pub use crate::root::{ IndexMetasForLeafSearch, SearchJob, check_all_index_metadata_found, jobs_to_leaf_request, root_search, search_plan, @@ -283,7 +287,7 @@ 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, diff --git a/quickwit/quickwit-search/src/list_terms.rs b/quickwit/quickwit-search/src/list_terms.rs index 3a72f26d595..5fb38124073 100644 --- a/quickwit/quickwit-search/src/list_terms.rs +++ b/quickwit/quickwit-search/src/list_terms.rs @@ -17,6 +17,7 @@ use std::ops::Bound; use std::sync::Arc; use anyhow::Context; +use bytesize::ByteSize; use futures::future::try_join_all; use itertools::{Either, Itertools}; use quickwit_common::pretty::PrettySample; @@ -326,14 +327,19 @@ pub async fn leaf_list_terms( splits: &[SplitIdAndFooterOffsets], ) -> Result { info!(split_offsets = ?PrettySample::new(splits, 5)); - let permit_sizes = splits.iter().map(|split| { - compute_initial_memory_allocation( - split, - searcher_context - .searcher_config - .warmup_single_split_initial_allocation, - ) - }); + let permit_sizes: Vec = splits + .iter() + .map(|split| { + compute_initial_memory_allocation( + split, + searcher_context + .searcher_config + .warmup_single_split_initial_allocation, + ) + }) + .collect(); + // allow offload to lambda + // https://github.com/quickwit-oss/quickwit/issues/6150 let permits = searcher_context .search_permit_provider .get_permits(permit_sizes) diff --git a/quickwit/quickwit-search/src/root.rs b/quickwit/quickwit-search/src/root.rs index b5b03882ec0..5d5610ca4d9 100644 --- a/quickwit/quickwit-search/src/root.rs +++ b/quickwit/quickwit-search/src/root.rs @@ -45,7 +45,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}; @@ -1237,7 +1237,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-search/src/search_permit_provider.rs b/quickwit/quickwit-search/src/search_permit_provider.rs index 3a4c0602326..3cb3693d8b2 100644 --- a/quickwit/quickwit-search/src/search_permit_provider.rs +++ b/quickwit/quickwit-search/src/search_permit_provider.rs @@ -40,9 +40,16 @@ pub struct SearchPermitProvider { } pub enum SearchPermitMessage { - Request { - permit_sender: oneshot::Sender>, + RequestWithOffload { permit_sizes: Vec, + /// Maximum number of pending requests. If granting permits all + /// requested permits would cause the number of pending requests to exceed this threshold, + /// some permits will be offloaded to Lambda. + offload_threshold: usize, + /// Channel to return the result message from the actor. + /// When offloading permits, the number of futures can be < to the number of requested + /// permits. + permit_resp_tx: oneshot::Sender>, }, UpdateMemory { memory_delta: i64, @@ -101,23 +108,38 @@ impl SearchPermitProvider { } } - /// Returns one permit future for each provided split metadata. + /// Returns permits for local splits /// - /// The permits returned are guaranteed to be resolved in order. In - /// addition, the permits are guaranteed to be resolved before permits - /// returned by subsequent calls to this function. + /// The returned futures are guaranteed to resolve in order. + pub async fn get_permits(&self, splits: Vec) -> Vec { + self.get_permits_with_offload(splits, usize::MAX).await + } + + /// 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. + /// + /// The returned futures are guaranteed to resolve in order. /// - /// The permit memory size is capped by per_permit_initial_memory_allocation. - pub async fn get_permits( + /// If `offload_threshold` is 0, all splits are offloaded. + /// If `offload_threshold` is usize::MAX, all splits are processed locally. + pub async fn get_permits_with_offload( &self, - splits: impl IntoIterator, + splits: Vec, + offload_threshold: usize, ) -> Vec { + if splits.is_empty() { + return Vec::new(); + } let (permit_sender, permit_receiver) = oneshot::channel(); let permit_sizes = splits.into_iter().map(|size| size.as_u64()).collect(); self.message_sender - .send(SearchPermitMessage::Request { - permit_sender, + .send(SearchPermitMessage::RequestWithOffload { + permit_resp_tx: permit_sender, permit_sizes, + offload_threshold, }) .expect("Receiver lives longer than sender"); permit_receiver @@ -177,7 +199,9 @@ impl PartialEq for LeafPermitRequest { impl Eq for LeafPermitRequest {} impl LeafPermitRequest { + // `permit_sizes` must not be empty. fn from_estimated_costs(permit_sizes: Vec) -> (Self, Vec) { + assert!(!permit_sizes.is_empty(), "permit_sizes must not be empty"); let mut permits = Vec::with_capacity(permit_sizes.len()); let mut single_split_permit_requests = Vec::with_capacity(permit_sizes.len()); for permit_size in permit_sizes { @@ -200,13 +224,11 @@ impl LeafPermitRequest { } fn pop_if_smaller_than(&mut self, max_size: u64) -> Option { - // IntoIter::as_slice() allows us to peek at the next element without consuming it - match self.single_split_permit_requests.as_slice().first() { - Some(request) if request.permit_size <= max_size => { - self.single_split_permit_requests.next() - } - _ => None, + let peeked_single_split_req = self.single_split_permit_requests.as_slice().first()?; + if peeked_single_split_req.permit_size > max_size { + return None; } + self.single_split_permit_requests.next() } fn is_empty(&self) -> bool { @@ -226,18 +248,35 @@ impl SearchPermitActor { fn handle_message(&mut self, msg: SearchPermitMessage) { match msg { - SearchPermitMessage::Request { - permit_sizes, - permit_sender, + SearchPermitMessage::RequestWithOffload { + mut permit_sizes, + permit_resp_tx: permit_sender, + offload_threshold, } => { - let (leaf_permit_request, permits) = + let current_pending = self + .permits_requests + .iter() + .map(|req| req.single_split_permit_requests.as_slice().len()) + .sum(); + // How many new splits can we accept locally before hitting the threshold. + let local_capacity = offload_threshold.saturating_sub(current_pending); + + // If this indeed truncates the permit_sizes vector, other splits will be offloaded + // to lambdas. + permit_sizes.truncate(local_capacity); + + // We special case here in order to avoid pushing empty request in the queue. + // (they would never be removed) + if permit_sizes.is_empty() { + let _ = permit_sender.send(Vec::new()); + return; + } + + let (leaf_permit_request, permit_futures) = LeafPermitRequest::from_estimated_costs(permit_sizes); self.permits_requests.push(leaf_permit_request); 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); + let _ = permit_sender.send(permit_futures); } SearchPermitMessage::UpdateMemory { memory_delta } => { if self.total_memory_allocated as i64 + memory_delta < 0 { @@ -276,6 +315,10 @@ impl SearchPermitActor { .checked_sub(self.total_memory_allocated)?; let mut peeked = self.permits_requests.peek_mut()?; + assert!( + !peeked.is_empty(), + "unexpected empty permits_requests present in the search permit provider queue" + ); if let Some(permit_request) = peeked.pop_if_smaller_than(available_memory) { if peeked.is_empty() { PeekMut::pop(peeked); @@ -380,7 +423,6 @@ impl Future for SearchPermitFuture { #[cfg(test)] mod tests { - use std::iter::repeat_n; use std::time::Duration; use futures::StreamExt; @@ -394,7 +436,7 @@ mod tests { let permit_provider = SearchPermitProvider::new(1, ByteSize::mb(100)); let mut all_futures = Vec::new(); let first_batch_of_permits = permit_provider - .get_permits(repeat_n(ByteSize::mb(10), 10)) + .get_permits(vec![ByteSize::mb(10); 10]) .await; assert_eq!(first_batch_of_permits.len(), 10); all_futures.extend( @@ -405,7 +447,7 @@ mod tests { ); let second_batch_of_permits = permit_provider - .get_permits(repeat_n(ByteSize::mb(10), 10)) + .get_permits(vec![ByteSize::mb(10); 10]) .await; assert_eq!(second_batch_of_permits.len(), 10); all_futures.extend( @@ -443,9 +485,7 @@ mod tests { async fn test_search_permit_order_with_concurrent_search() { let permit_provider = SearchPermitProvider::new(4, ByteSize::mb(100)); let mut all_futures = Vec::new(); - let first_batch_of_permits = permit_provider - .get_permits(repeat_n(ByteSize::mb(10), 8)) - .await; + let first_batch_of_permits = permit_provider.get_permits(vec![ByteSize::mb(10); 8]).await; assert_eq!(first_batch_of_permits.len(), 8); all_futures.extend( first_batch_of_permits @@ -454,9 +494,7 @@ mod tests { .map(move |(i, fut)| ((1, i), fut)), ); - let second_batch_of_permits = permit_provider - .get_permits(repeat_n(ByteSize::mb(10), 2)) - .await; + let second_batch_of_permits = permit_provider.get_permits(vec![ByteSize::mb(10); 2]).await; all_futures.extend( second_batch_of_permits .into_iter() @@ -464,9 +502,7 @@ mod tests { .map(move |(i, fut)| ((2, i), fut)), ); - let third_batch_of_permits = permit_provider - .get_permits(repeat_n(ByteSize::mb(10), 6)) - .await; + let third_batch_of_permits = permit_provider.get_permits(vec![ByteSize::mb(10); 6]).await; all_futures.extend( third_batch_of_permits .into_iter() @@ -518,7 +554,7 @@ mod tests { .next() .unwrap(); let permit_fut2 = permit_provider - .get_permits([ByteSize::mb(10)]) + .get_permits(vec![ByteSize::mb(10)]) .await .into_iter() .next() @@ -529,7 +565,7 @@ mod tests { assert_eq!(*permit_provider.actor_stopped.borrow(), false); let _permit_fut3 = permit_provider - .get_permits([ByteSize::mb(10)]) + .get_permits(vec![ByteSize::mb(10)]) .await .into_iter() .next() @@ -553,7 +589,7 @@ mod tests { async fn test_memory_budget() { let permit_provider = SearchPermitProvider::new(100, ByteSize::mb(100)); let mut permit_futs = permit_provider - .get_permits(repeat_n(ByteSize::mb(10), 14)) + .get_permits(vec![ByteSize::mb(10); 14]) .await; let mut remaining_permit_futs = permit_futs.split_off(10).into_iter(); assert_eq!(remaining_permit_futs.len(), 4); @@ -579,12 +615,85 @@ mod tests { try_get(next_permit_fut).await.unwrap(); } + #[tokio::test] + async fn test_get_permits_with_offload_threshold_max_returns_all() { + let permit_provider = SearchPermitProvider::new(100, ByteSize::mb(100)); + let permits = permit_provider + .get_permits_with_offload(vec![ByteSize::mb(1); 8], usize::MAX) + .await; + assert_eq!(permits.len(), 8); + } + + #[tokio::test] + async fn test_get_permits_with_offload_threshold_zero_returns_none() { + let permit_provider = SearchPermitProvider::new(100, ByteSize::mb(100)); + let permits = permit_provider + .get_permits_with_offload(vec![ByteSize::mb(1); 5], 0) + .await; + assert!(permits.is_empty()); + } + + #[tokio::test] + async fn test_get_permits_with_offload_truncates_to_threshold() { + let permit_provider = SearchPermitProvider::new(100, ByteSize::mb(100)); + let permits = permit_provider + .get_permits_with_offload(vec![ByteSize::mb(1); 10], 4) + .await; + assert_eq!(permits.len(), 4); + } + + #[tokio::test] + async fn test_get_permits_with_offload_futures_resolve_in_order() { + // We use a search permit provider with a capacity of 1 to make sure that the permits are + // resolved in order. + let permit_provider = SearchPermitProvider::new(1, ByteSize::mb(100)); + let permits = permit_provider + .get_permits_with_offload(vec![ByteSize::mb(1); 4], 10) + .await; + assert_eq!(permits.len(), 4); + let mut futs: Vec<_> = permits + .into_iter() + .enumerate() + .map(|(i, permit_fut)| async move { + permit_fut.await; + i + }) + .collect(); + futs.shuffle(&mut rand::rng()); + let mut join_set = JoinSet::new(); + for fut in futs { + join_set.spawn(fut); + } + let mut results = Vec::new(); + while let Some(result) = join_set.join_next().await { + results.push(result.unwrap()); + } + assert_eq!(results, vec![0, 1, 2, 3]); + } + + #[tokio::test] + async fn test_get_permits_with_offload_pending_consumed_frees_capacity() { + let permit_provider = SearchPermitProvider::new(100, ByteSize::mb(100)); + // First call: 4 splits, threshold 6. + let first_permits = permit_provider + .get_permits_with_offload(vec![ByteSize::mb(1); 4], 6) + .await; + assert_eq!(first_permits.len(), 4); + // Consume all permits from the first batch (they resolve and get dropped). + for permit_fut in first_permits { + let _permit = permit_fut.await; + } + // Second call: the consumed permits no longer count as pending. + let second_permits = permit_provider + .get_permits_with_offload(vec![ByteSize::mb(1); 5], 6) + .await; + assert_eq!(second_permits.len(), 5); + } + #[tokio::test] async fn test_warmup_slot() { let permit_provider = SearchPermitProvider::new(10, ByteSize::mb(100)); - let mut permit_futs = permit_provider - .get_permits(repeat_n(ByteSize::mb(1), 16)) - .await; + let mut permit_futs = permit_provider.get_permits(vec![ByteSize::mb(1); 16]).await; let mut remaining_permit_futs = permit_futs.split_off(10).into_iter(); assert_eq!(remaining_permit_futs.len(), 6); // we should be able to obtain 10 permits right away diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index cf2c0699b81..a82edf2d9fc 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -33,6 +33,7 @@ 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}; @@ -190,7 +191,7 @@ impl SearchService for SearchServiceImpl { tracked: multi_index_leaf_search( self.searcher_context.clone(), leaf_search_request, - &self.storage_resolver, + self.storage_resolver.clone(), ), start: Instant::now(), targeted_splits: num_splits, @@ -415,7 +416,10 @@ pub struct SearcherContext { /// List fields cache. Caches the list fields response for a given split. pub list_fields_cache: ListFieldsCache, /// The aggregation limits are passed to limit the memory usage. + /// This object is shared across all request. pub aggregation_limit: AggregationLimitsGuard, + /// Optional Lambda invoker for offloading leaf search to serverless functions. + pub lambda_invoker: Option>, } impl std::fmt::Debug for SearcherContext { @@ -431,11 +435,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, @@ -464,6 +472,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..61751e3d253 100644 --- a/quickwit/quickwit-search/src/tests.rs +++ b/quickwit/quickwit-search/src/tests.rs @@ -1029,9 +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)); - - let agg_limits = searcher_context.get_aggregation_limits(); + Arc::new(SearcherContext::new(SearcherConfig::default(), None, None)); let search_response = single_doc_mapping_leaf_search( searcher_context, @@ -1039,7 +1037,6 @@ async fn test_search_util(test_sandbox: &TestSandbox, query: &str) -> Vec { test_sandbox.storage(), splits_offsets, test_sandbox.doc_mapper(), - agg_limits, ) .await .unwrap(); @@ -1669,7 +1666,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/Cargo.toml b/quickwit/quickwit-serve/Cargo.toml index 363065a3403..16f8a36002d 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-client = { 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..561c8b4d51a 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -630,9 +630,23 @@ pub async fn serve_quickwit( None }; + // Initialize Lambda invoker if enabled and searcher service is running + let lambda_invoker_opt = if node_config.is_service_enabled(QuickwitService::Searcher) { + if let Some(lambda_config) = &node_config.searcher_config.lambda { + info!("initializing AWS Lambda invoker for search"); + let invoker = quickwit_lambda_client::try_get_or_deploy_invoker(lambda_config).await?; + Some(invoker) + } else { + None + } + } 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( @@ -1017,6 +1031,7 @@ async fn setup_searcher( ) -> anyhow::Result<(SearchJobPlacer, Arc)> { let searcher_pool = SearcherPool::default(); let search_job_placer = SearchJobPlacer::new(searcher_pool.clone()); + let search_service = start_searcher_service( metastore, storage_resolver, @@ -1553,7 +1568,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(); diff --git a/quickwit/quickwit-storage/src/cache/memory_sized_cache.rs b/quickwit/quickwit-storage/src/cache/memory_sized_cache.rs index 305966d3c04..707d633501f 100644 --- a/quickwit/quickwit-storage/src/cache/memory_sized_cache.rs +++ b/quickwit/quickwit-storage/src/cache/memory_sized_cache.rs @@ -218,4 +218,14 @@ mod tests { cache.put("hello.seg", data); assert_eq!(cache.get(&"hello.seg").unwrap(), &b"werwer"[..]); } + + #[test] + fn test_cache_no_cache() { + let cache = + MemorySizedCache::from_config(&CacheConfig::no_cache(), &CACHE_METRICS_FOR_TESTS); + assert!(cache.get(&"hello.seg").is_none()); + let data = OwnedBytes::new(&b"werwer"[..]); + cache.put("hello.seg", data); + assert!(cache.get(&"hello.seg").is_none()); + } } From 7808c543bd4136d6d7ed1f823a8a5515d61b842c Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Mon, 16 Feb 2026 17:50:59 +0100 Subject: [PATCH 2/6] CR comment, checking sha256 of downloaded file --- docs/configuration/lambda-config.md | 25 +++- quickwit/Cargo.lock | 14 +- quickwit/Cargo.toml | 2 +- quickwit/quickwit-lambda-client/Cargo.toml | 4 +- quickwit/quickwit-lambda-client/build.rs | 125 ++++++++++-------- quickwit/quickwit-lambda-client/src/deploy.rs | 112 ++++++++++------ quickwit/quickwit-search/src/leaf.rs | 2 +- quickwit/quickwit-search/src/list_terms.rs | 3 +- .../src/search_permit_provider.rs | 2 +- quickwit/quickwit-serve/src/lib.rs | 1 + 10 files changed, 178 insertions(+), 112 deletions(-) diff --git a/docs/configuration/lambda-config.md b/docs/configuration/lambda-config.md index 4803d1a2418..5b2184c32fd 100644 --- a/docs/configuration/lambda-config.md +++ b/docs/configuration/lambda-config.md @@ -135,7 +135,7 @@ If using `auto_deploy`, additional permissions are required for deployment: ### Lambda execution role -The Lambda function requires an execution role with S3 read access to your index data. CloudWatch logging permissions are not required. +The Lambda function requires an execution role with S3 read access to your index data. Example policy: @@ -169,6 +169,29 @@ The execution role must also have a trust policy allowing Lambda to assume it: } ``` +## CloudWatch logging + +The Lambda function emits structured logs (JSON) to stdout. To have these logs captured by CloudWatch, add the following iam permissions to the Lambda execution role: + +```json +{ + "Version": "2012-10-17", + "Statement": [ + { + "Effect": "Allow", + "Action": [ + "logs:CreateLogGroup", + "logs:CreateLogStream", + "logs:PutLogEvents" + ], + "Resource": "arn:aws:logs:*:*:*" + } + ] +} +``` + +No additional configuration is needed on the Quickwit side. + ## Versioning Quickwit uses content-based versioning for Lambda: diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 53261a6b54b..d5e1d4a9d84 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7114,10 +7114,10 @@ dependencies = [ "quickwit-search", "quickwit-storage", "serde_json", + "sha2", "tokio", "tracing", "ureq", - "zip", ] [[package]] @@ -11550,18 +11550,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 78655a74b0d..3ffdcb2f5a6 100644 --- a/quickwit/Cargo.toml +++ b/quickwit/Cargo.toml @@ -226,6 +226,7 @@ serde_qs = { version = "0.15" } serde_with = "3.16" serde_yaml = "0.9" serial_test = { version = "3.2", features = ["file_locks"] } +sha2 = "0.10" siphasher = "1.0" smallvec = "1" sqlx = { version = "0.8", features = [ @@ -303,7 +304,6 @@ vrl = { version = "0.29", default-features = false, features = [ ] } warp = { version = "0.4", features = ["server", "test"] } 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-lambda-client/Cargo.toml b/quickwit/quickwit-lambda-client/Cargo.toml index c96f39e46de..257172068b3 100644 --- a/quickwit/quickwit-lambda-client/Cargo.toml +++ b/quickwit/quickwit-lambda-client/Cargo.toml @@ -16,6 +16,7 @@ async-trait = { workspace = true } aws-config = { workspace = true } aws-sdk-lambda = { workspace = true } base64 = { workspace = true } +md5 = { workspace = true } prost = { workspace = true } serde_json = { workspace = true } once_cell = { workspace = true } @@ -40,6 +41,5 @@ tokio = { workspace = true, features = ["test-util", "macros"] } quickwit-storage = { workspace = true, features = ["testsuite"] } [build-dependencies] -md5 = { workspace = true } +sha2 = { workspace = true } ureq = { workspace = true } -zip = { workspace = true, default-features = false, features = ["deflate"] } diff --git a/quickwit/quickwit-lambda-client/build.rs b/quickwit/quickwit-lambda-client/build.rs index 193e7a054fa..6223ed7668c 100644 --- a/quickwit/quickwit-lambda-client/build.rs +++ b/quickwit/quickwit-lambda-client/build.rs @@ -20,12 +20,18 @@ //! The Lambda binary is built separately in CI and published as a GitHub release. use std::env; -use std::path::PathBuf; +use std::path::{Path, PathBuf}; + +use sha2::{Digest, Sha256}; /// 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 = "https://github.com/quickwit-oss/quickwit/releases/download/lambda-ff6fdfa5/quickwit-aws-lambda--aarch64.zip"; +/// Expected SHA256 hash of the Lambda zip artifact. +/// Must be updated alongside LAMBDA_ZIP_URL when a new Lambda binary is released. +const LAMBDA_ZIP_SHA256: &str = "fa940f44178e28460c21e44bb2610b776542b9b97db66a53bc65b10cad653b90"; + /// AWS Lambda direct upload limit is 50MB. /// Larger artifacts must be uploaded via S3. const MAX_LAMBDA_ZIP_SIZE: usize = 50 * 1024 * 1024; @@ -35,73 +41,82 @@ fn main() { let out_dir = PathBuf::from(env::var("OUT_DIR").expect("OUT_DIR not set")); let zip_path = out_dir.join("lambda_bootstrap.zip"); - let url_marker_path = out_dir.join("lambda_bootstrap.url"); - - // Check if we already have the zip from the same URL - let needs_download = if zip_path.try_exists().unwrap() && url_marker_path.try_exists().unwrap() - { - let cached_url = std::fs::read_to_string(&url_marker_path).unwrap_or_default(); - cached_url.trim() != LAMBDA_ZIP_URL - } else { - true - }; - - let lambda_zip_payload: Vec = if needs_download { - println!( - "cargo:warning=Downloading Lambda zip from: {}", - LAMBDA_ZIP_URL - ); - let data: Vec = - download_lambda_zip(LAMBDA_ZIP_URL).expect("failed to download lambda zip"); - std::fs::write(&zip_path, &data).expect("Failed to write zip file"); - std::fs::write(&url_marker_path, LAMBDA_ZIP_URL).expect("Failed to write URL marker"); - println!( - "cargo:warning=Downloaded Lambda zip to {:?} ({} bytes)", - zip_path, - data.len() - ); - data - } else { - println!("Using cached Lambda zip from {:?}", zip_path); - std::fs::read(&zip_path).expect("Failed to read cached zip file") - }; - // Compute MD5 hash of the zip and export as environment variable. + fetch_lambda_zip(&zip_path); + + // Export first 8 hex chars of the SHA256 as environment variable. // This is used to create a unique qualifier for Lambda versioning. - let digest = md5::compute(&lambda_zip_payload); - let hash_short = &format!("{:x}", digest)[..8]; // First 8 hex chars + let hash_short = &LAMBDA_ZIP_SHA256[..8]; println!("cargo:rustc-env=LAMBDA_BINARY_HASH={}", hash_short); - println!("Lambda binary hash (short): {}", hash_short); + println!("lambda binary hash (short): {}", hash_short); } -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? +/// Fetch the Lambda zip and save it to `local_cache_path`. +/// +/// If a cached file already exists with the correct SHA256, this is a no-op. +/// If the hash doesn't match (stale artifact), the file is deleted and re-downloaded. +/// If no cached file exists, the artifact is downloaded fresh. +/// +/// This function panics if a problem happens. +fn fetch_lambda_zip(local_cache_path: &Path) { + // Try the cache first. + if let Ok(data) = std::fs::read(local_cache_path) { + let actual_hash = sha256_hex(&data); + if actual_hash == LAMBDA_ZIP_SHA256 { + println!("using cached Lambda zip from {:?}", local_cache_path); + return; + } + println!("cargo:warning=cached Lambda zip has wrong SHA256, re-downloading"); + std::fs::remove_file(local_cache_path).expect("failed to delete stale cached zip"); + } + + // Download from the remote URL. + println!( + "cargo:warning=downloading Lambda zip from: {}", + LAMBDA_ZIP_URL + ); + let data = download_lambda_zip(LAMBDA_ZIP_URL).expect("failed to download Lambda zip"); + + // Verify SHA256 BEFORE writing to disk. + let actual_hash = sha256_hex(&data); + if actual_hash != LAMBDA_ZIP_SHA256 { + panic!( + "SHA256 mismatch for Lambda zip!\n expected: {LAMBDA_ZIP_SHA256}\n actual: \ + {actual_hash}\nThe artifact at {LAMBDA_ZIP_URL} may have been tampered with." + ); + } + + std::fs::write(local_cache_path, &data).expect("failed to write zip file"); + println!( + "cargo:warning=downloaded Lambda zip to {:?} ({} bytes)", + local_cache_path, + data.len() + ); +} + +fn sha256_hex(data: &[u8]) -> String { + format!("{:x}", Sha256::digest(data)) +} + +fn download_lambda_zip(url: &str) -> Result, String> { + let response = ureq::get(url) + .call() + .map_err(|err| format!("HTTP request failed: {err}"))?; + // Set limit higher than MAX_LAMBDA_ZIP_SIZE so we can detect oversized artifacts. + 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()?; + .limit(MAX_LAMBDA_ZIP_SIZE as u64 + 1) + .read_to_vec() + .map_err(|err| format!("failed to read response body: {err}"))?; if data.len() > MAX_LAMBDA_ZIP_SIZE { - panic!( + return Err(format!( "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 - ); + )); } - validate_zip(&data)?; Ok(data) } - -fn validate_zip(data: &[u8]) -> Result<(), Box> { - let cursor = std::io::Cursor::new(data); - let mut archive = zip::ZipArchive::new(cursor)?; - // Verify we can read all entries (checks CRC for each file) - for i in 0..archive.len() { - let mut file = archive.by_index(i)?; - std::io::copy(&mut file, &mut std::io::sink())?; - } - Ok(()) -} diff --git a/quickwit/quickwit-lambda-client/src/deploy.rs b/quickwit/quickwit-lambda-client/src/deploy.rs index e7890bfc059..4fc1b637596 100644 --- a/quickwit/quickwit-lambda-client/src/deploy.rs +++ b/quickwit/quickwit-lambda-client/src/deploy.rs @@ -37,7 +37,7 @@ use aws_sdk_lambda::types::{ }; use quickwit_config::{LambdaConfig, LambdaDeployConfig}; use quickwit_search::LambdaLeafSearchInvoker; -use tracing::{debug, info, warn}; +use tracing::{debug, info}; use crate::invoker::create_lambda_invoker_for_version; @@ -68,8 +68,31 @@ fn lambda_qualifier() -> &'static str { } /// Returns the version description for our qualifier. -fn version_description() -> String { - format!("{}{}", VERSION_DESCRIPTION_PREFIX, lambda_qualifier()) +/// +/// We also pass the deploy config, as we want the function to be redeployed +/// if the deployed is changed. +fn version_description(deploy_config_opt: Option<&LambdaDeployConfig>) -> String { + if let Some(deploy_config) = deploy_config_opt { + let memory_size_mib = deploy_config.memory_size.as_mib() as u64; + let execution_role_arn_digest: String = format!( + "{:x}", + md5::compute(deploy_config.execution_role_arn.as_bytes()) + ); + format!( + "{}_{}_{}_{}s_{}", + VERSION_DESCRIPTION_PREFIX, + lambda_qualifier(), + memory_size_mib, + deploy_config.invocation_timeout_secs, + &execution_role_arn_digest[..5] + ) + } else { + format!( + "{}_{}_nodeploy", + VERSION_DESCRIPTION_PREFIX, + lambda_qualifier() + ) + } } /// Get or deploy the Lambda function and return an invoker. @@ -89,12 +112,12 @@ pub async fn try_get_or_deploy_invoker( 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(); + let target_description = version_description(lambda_config.auto_deploy.as_ref()); info!( function_name = %function_name, qualifier = %lambda_qualifier(), - "Looking for Lambda function version" + "looking for Lambda function version" ); let version = find_or_deploy_version( @@ -105,7 +128,7 @@ pub async fn try_get_or_deploy_invoker( ) .await?; - // Step 3: Spawn background garbage collection (best effort, non-blocking) + // 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(); @@ -113,14 +136,14 @@ pub async fn try_get_or_deploy_invoker( 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"); + info!(error = %e, "failed to garbage collect old Lambda versions"); } }); - // Step 4: Create and return the invoker + // Create and return the invoker let invoker = create_lambda_invoker_for_version(function_name.clone(), version) .await - .context("Failed to create Lambda invoker")?; + .context("failed to create Lambda invoker")?; info!("created the lambda invoker"); @@ -149,7 +172,7 @@ async fn find_or_deploy_version( let deploy_config = deploy_config.with_context(|| { format!( - "No Lambda version found with description '{}' and auto_deploy is not configured. \ + "no Lambda version found with description '{}' and auto_deploy is not configured. \ Either deploy the Lambda function manually or enable auto_deploy.", target_description ) @@ -237,9 +260,12 @@ async fn deploy_lambda_function( // This looks overly complicated but this is not AI slop. // The AWS API forces us to go through a bunch of hoops to update our function // in a safe manner. + let description = version_description(Some(deploy_config)); // Fast path: if the function does not exist, we can create and publish the function atomically. - if let Some(version) = try_create_function(client, function_name, deploy_config).await? { + if let Some(version) = + try_create_function(client, function_name, deploy_config, &description).await? + { return Ok(version); } @@ -247,7 +273,7 @@ async fn deploy_lambda_function( // This will create or update a version called "$LATEST" (that's the actual string) // // We cannot directly publish here, because updating the function code does not allow - // use to pass a different description. + // us to pass a different description. let code_sha256 = update_function_code(client, function_name).await?; // We can now publish that new uploaded version. @@ -255,7 +281,7 @@ async fn deploy_lambda_function( // us to publish a different version. // // Publishing will create an actual version (a number as a string) and return it. - publish_version(client, function_name, &code_sha256).await + publish_version(client, function_name, &code_sha256, &description).await } /// Try to create the Lambda function with `publish=true`. @@ -266,19 +292,19 @@ async fn try_create_function( client: &LambdaClient, function_name: &str, deploy_config: &LambdaDeployConfig, + description: &str, ) -> anyhow::Result> { let memory_size_mb = deploy_config .memory_size .as_u64() .div_ceil(1024u64 * 1024u64) 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" + "attempting to create Lambda function" ); let function_code = FunctionCode::builder() @@ -291,7 +317,7 @@ async fn try_create_function( .runtime(Runtime::Providedal2023) .role(&deploy_config.execution_role_arn) .handler("bootstrap") - .description(&description) + .description(description) .code(function_code) .architectures(Architecture::Arm64) .memory_size(memory_size_mb) @@ -306,24 +332,24 @@ async fn try_create_function( Ok(output) => { let version = output .version() - .ok_or_else(|| anyhow!("Created function has no version number"))? + .ok_or_else(|| anyhow!("created function has no version number"))? .to_string(); info!( function_name = %function_name, version = %version, - "Lambda function created and published" + "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" + "lambda function already exists" ); Ok(None) } Err(e) => Err(anyhow!( - "Failed to create Lambda function '{}': {}", + "failed to create Lambda function '{}': {}", function_name, e )), @@ -351,7 +377,7 @@ async fn update_function_code( .architectures(Architecture::Arm64) .send() .await - .context("Failed to update Lambda function code")?; + .context("failed to update Lambda function code")?; let code_sha256 = response .code_sha256() @@ -374,9 +400,8 @@ async fn publish_version( client: &LambdaClient, function_name: &str, code_sha256: &str, + description: &str, ) -> anyhow::Result { - let description = version_description(); - info!( function_name = %function_name, description = %description, @@ -386,18 +411,18 @@ async fn publish_version( let publish_response = client .publish_version() .function_name(function_name) - .description(&description) + .description(description) .code_sha256(code_sha256) .send() .await .context( - "Failed to publish Lambda version (code_sha256 mismatch means a concurrent deploy \ + "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")? + .context("published version has no version number")? .to_string(); info!( @@ -432,7 +457,7 @@ async fn wait_for_function_ready(client: &LambdaClient, function_name: &str) -> .function_name(function_name) .send() .await - .context("Failed to get function status")?; + .context("failed to get function status")?; let Some(config) = response.configuration() else { continue; @@ -440,9 +465,9 @@ async fn wait_for_function_ready(client: &LambdaClient, function_name: &str) -> // Check for terminal failure states. if config.state() == Some(&State::Failed) { - let reason = config.state_reason().unwrap_or("Unknown reason"); + let reason = config.state_reason().unwrap_or("unknown reason"); anyhow::bail!( - "Lambda function '{}' is in Failed state: {}", + "lambda function '{}' is in Failed state: {}", function_name, reason ); @@ -455,9 +480,9 @@ async fn wait_for_function_ready(client: &LambdaClient, function_name: &str) -> if last_update_status == &LastUpdateStatus::Failed { let reason = config .last_update_status_reason() - .unwrap_or("Unknown reason"); + .unwrap_or("unknown reason"); anyhow::bail!( - "Lambda function '{}' last update failed: {}", + "lambda function '{}' last update failed: {}", function_name, reason ); @@ -484,7 +509,7 @@ async fn wait_for_function_ready(client: &LambdaClient, function_name: &str) -> } anyhow::bail!( - "Lambda function '{}' did not become ready within {} seconds", + "lambda function '{}' did not become ready within {} seconds", function_name, MAX_WAIT_ATTEMPTS as u64 * WAIT_INTERVAL.as_secs() ) @@ -512,7 +537,7 @@ async fn garbage_collect_old_versions( let response = request .send() .await - .context("Failed to list Lambda versions for garbage collection")?; + .context("failed to list Lambda versions for garbage collection")?; for version in response.versions() { let Some(version_str) = version.version() else { @@ -570,11 +595,11 @@ async fn garbage_collect_old_versions( .send() .await { - warn!( + info!( function_name = %function_name, version = %version, error = %e, - "Failed to delete old Lambda version" + "failed to delete old Lambda version" ); } } @@ -629,6 +654,19 @@ mod tests { } } + fn test_description() -> String { + version_description(None) + } + + #[test] + fn test_version_description() { + let lambda_deploy_config = test_deploy_config(); + let description = version_description(Some(&lambda_deploy_config)); + assert!(description.ends_with("_60s_6c3b2")); + let description = version_description(None); + assert!(description.ends_with("_nodeploy")); + } + // --- find_matching_version tests --- #[tokio::test] @@ -708,7 +746,7 @@ mod tests { let client = mock_client!(aws_sdk_lambda, [&rule]); let config = test_deploy_config(); - let result = try_create_function(&client, "my-fn", &config) + let result = try_create_function(&client, "my-fn", &config, &test_description()) .await .unwrap(); assert_eq!(result, Some("1".to_string())); @@ -724,7 +762,7 @@ mod tests { let client = mock_client!(aws_sdk_lambda, [&rule]); let config = test_deploy_config(); - let result = try_create_function(&client, "my-fn", &config) + let result = try_create_function(&client, "my-fn", &config, &test_description()) .await .unwrap(); assert_eq!(result, None); diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index ccee5d6e27a..9bb32f7c2c3 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -1687,7 +1687,7 @@ async fn run_local_search_tasks( task_id_to_split_id_map.insert(handle.id(), split_id); } - // Step 5: Await all local tasks. + // Await all local tasks. let mut split_search_join_errors: Vec<(String, JoinError)> = Vec::new(); while let Some(leaf_search_join_result) = split_search_joinset.join_next().await { diff --git a/quickwit/quickwit-search/src/list_terms.rs b/quickwit/quickwit-search/src/list_terms.rs index 5fb38124073..5c69972728e 100644 --- a/quickwit/quickwit-search/src/list_terms.rs +++ b/quickwit/quickwit-search/src/list_terms.rs @@ -338,7 +338,8 @@ pub async fn leaf_list_terms( ) }) .collect(); - // allow offload to lambda + // We have added offloading leaf search to lambdas, but not for list_terms yet. + // TODO (Add it) // https://github.com/quickwit-oss/quickwit/issues/6150 let permits = searcher_context .search_permit_provider diff --git a/quickwit/quickwit-search/src/search_permit_provider.rs b/quickwit/quickwit-search/src/search_permit_provider.rs index 3cb3693d8b2..cc59958b0d1 100644 --- a/quickwit/quickwit-search/src/search_permit_provider.rs +++ b/quickwit/quickwit-search/src/search_permit_provider.rs @@ -42,7 +42,7 @@ pub struct SearchPermitProvider { pub enum SearchPermitMessage { RequestWithOffload { permit_sizes: Vec, - /// Maximum number of pending requests. If granting permits all + /// Maximum number of pending requests. If granting all /// requested permits would cause the number of pending requests to exceed this threshold, /// some permits will be offloaded to Lambda. offload_threshold: usize, diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 561c8b4d51a..dccaecd3912 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -634,6 +634,7 @@ pub async fn serve_quickwit( let lambda_invoker_opt = if node_config.is_service_enabled(QuickwitService::Searcher) { if let Some(lambda_config) = &node_config.searcher_config.lambda { info!("initializing AWS Lambda invoker for search"); + warn!("offloading to lambda is EXPERIMENTAL. Use at your own risk"); let invoker = quickwit_lambda_client::try_get_or_deploy_invoker(lambda_config).await?; Some(invoker) } else { From 6627ae79fa08d71dab8132116f119ef924bd82c6 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Wed, 18 Feb 2026 11:41:05 +0100 Subject: [PATCH 3/6] Changes in the lambda offloading Lambda now details (but does not retry) which split failed or was successful. Leaf cache lambda individual split result, (keep track of the rewritten) request to do so). Lambda handler returned split-id named results in any order. --- .../quickwit-lambda-client/src/invoker.rs | 18 ++--- quickwit/quickwit-lambda-client/src/lib.rs | 2 +- .../src/bin/leaf_search.rs | 6 +- .../quickwit-lambda-server/src/handler.rs | 77 +++++++++++-------- quickwit/quickwit-lambda-server/src/lib.rs | 2 +- .../protos/quickwit/search.proto | 26 +++++-- .../src/codegen/quickwit/quickwit.search.rs | 39 ++++++++-- quickwit/quickwit-search/src/invoker.rs | 9 ++- quickwit/quickwit-search/src/leaf.rs | 77 +++++++++++++++---- 9 files changed, 180 insertions(+), 76 deletions(-) diff --git a/quickwit/quickwit-lambda-client/src/invoker.rs b/quickwit/quickwit-lambda-client/src/invoker.rs index 9c4204be346..7f29a64ba60 100644 --- a/quickwit/quickwit-lambda-client/src/invoker.rs +++ b/quickwit/quickwit-lambda-client/src/invoker.rs @@ -21,8 +21,8 @@ use aws_sdk_lambda::primitives::Blob; use aws_sdk_lambda::types::InvocationType; use base64::prelude::*; use prost::Message; -use quickwit_lambda_server::{LeafSearchRequestPayload, LeafSearchResponsePayload}; -use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse, LeafSearchResponses}; +use quickwit_lambda_server::{LambdaSearchRequestPayload, LambdaSearchResponsePayload}; +use quickwit_proto::search::{LambdaSearchResponses, LambdaSingleSplitResult, LeafSearchRequest}; use quickwit_search::{LambdaLeafSearchInvoker, SearchError}; use tracing::{debug, info, instrument}; @@ -85,7 +85,7 @@ impl LambdaLeafSearchInvoker for AwsLambdaInvoker { async fn invoke_leaf_search( &self, request: LeafSearchRequest, - ) -> Result, SearchError> { + ) -> Result, SearchError> { let start = std::time::Instant::now(); let result = self.invoke_leaf_search_inner(request).await; @@ -109,10 +109,10 @@ impl AwsLambdaInvoker { async fn invoke_leaf_search_inner( &self, request: LeafSearchRequest, - ) -> Result, SearchError> { + ) -> Result, SearchError> { // Serialize request to protobuf bytes, then base64 encode let request_bytes = request.encode_to_vec(); - let payload = LeafSearchRequestPayload { + let payload = LambdaSearchRequestPayload { payload: BASE64_STANDARD.encode(&request_bytes), }; @@ -164,7 +164,7 @@ impl AwsLambdaInvoker { .leaf_search_response_payload_size_bytes .observe(response_payload.as_ref().len() as f64); - let lambda_response: LeafSearchResponsePayload = + let lambda_response: LambdaSearchResponsePayload = serde_json::from_slice(response_payload.as_ref()) .map_err(|e| SearchError::Internal(format!("json deserialization error: {}", e)))?; @@ -172,14 +172,14 @@ impl AwsLambdaInvoker { .decode(&lambda_response.payload) .map_err(|e| SearchError::Internal(format!("base64 decode error: {}", e)))?; - let leaf_responses = LeafSearchResponses::decode(&response_bytes[..]) + let leaf_responses = LambdaSearchResponses::decode(&response_bytes[..]) .map_err(|e| SearchError::Internal(format!("protobuf decode error: {}", e)))?; debug!( - num_responses = leaf_responses.responses.len(), + num_results = leaf_responses.split_results.len(), "lambda invocation completed" ); - Ok(leaf_responses.responses) + Ok(leaf_responses.split_results) } } diff --git a/quickwit/quickwit-lambda-client/src/lib.rs b/quickwit/quickwit-lambda-client/src/lib.rs index f1bee07cb5b..aebf264df8c 100644 --- a/quickwit/quickwit-lambda-client/src/lib.rs +++ b/quickwit/quickwit-lambda-client/src/lib.rs @@ -34,4 +34,4 @@ mod metrics; pub use deploy::try_get_or_deploy_invoker; pub use metrics::LAMBDA_METRICS; // Re-export payload types from server crate for convenience -pub use quickwit_lambda_server::{LeafSearchRequestPayload, LeafSearchResponsePayload}; +pub use quickwit_lambda_server::{LambdaSearchRequestPayload, LambdaSearchResponsePayload}; diff --git a/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs b/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs index 6a4dacab392..0ccdbc741e0 100644 --- a/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs +++ b/quickwit/quickwit-lambda-server/src/bin/leaf_search.rs @@ -17,7 +17,9 @@ use std::sync::Arc; use lambda_runtime::{Error, LambdaEvent, service_fn}; -use quickwit_lambda_server::{LambdaSearcherContext, LeafSearchRequestPayload, handle_leaf_search}; +use quickwit_lambda_server::{ + LambdaSearchRequestPayload, LambdaSearcherContext, handle_leaf_search, +}; use tracing::info; use tracing_subscriber::EnvFilter; @@ -36,7 +38,7 @@ async fn main() -> Result<(), Error> { // Run the Lambda handler lambda_runtime::run(service_fn( - |event: LambdaEvent| { + |event: LambdaEvent| { let ctx = Arc::clone(&context); async move { let (payload, _event_ctx) = event.into_parts(); diff --git a/quickwit/quickwit-lambda-server/src/handler.rs b/quickwit/quickwit-lambda-server/src/handler.rs index 15387975a3c..5312f8337bd 100644 --- a/quickwit/quickwit-lambda-server/src/handler.rs +++ b/quickwit/quickwit-lambda-server/src/handler.rs @@ -14,8 +14,9 @@ use base64::prelude::*; use prost::Message; +use quickwit_proto::search::lambda_single_split_result::Outcome; use quickwit_proto::search::{ - LeafSearchRequest, LeafSearchResponse, LeafSearchResponses, SplitIdAndFooterOffsets, + LambdaSearchResponses, LambdaSingleSplitResult, LeafSearchRequest, SplitIdAndFooterOffsets, }; use quickwit_search::leaf::multi_index_leaf_search; use serde::{Deserialize, Serialize}; @@ -26,27 +27,28 @@ use crate::error::{LambdaError, LambdaResult}; /// Payload for leaf search Lambda invocation. #[derive(Debug, Serialize, Deserialize)] -pub struct LeafSearchRequestPayload { +pub struct LambdaSearchRequestPayload { /// Base64-encoded serialized LeafSearchRequest protobuf. pub payload: String, } /// Response from leaf search Lambda invocation. #[derive(Debug, Serialize, Deserialize)] -pub struct LeafSearchResponsePayload { - /// Base64-encoded serialized `LeafSearchResponses` protobuf (one per split). +pub struct LambdaSearchResponsePayload { + /// Base64-encoded serialized `LambdaSearchResponses` 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. +/// Returns one `LambdaSingleSplitResult` per split, each tagged with its +/// split_id. Individual split failures are reported per-split rather than +/// failing the entire invocation, so the caller can retry only failed splits. #[instrument(skip(ctx), fields(request_id))] pub async fn handle_leaf_search( - event: LeafSearchRequestPayload, + event: LambdaSearchRequestPayload, ctx: &LambdaSearcherContext, -) -> LambdaResult { +) -> LambdaResult { // Decode base64 payload let request_bytes = BASE64_STANDARD .decode(&event.payload) @@ -70,11 +72,12 @@ pub async fn handle_leaf_search( let num_splits = all_splits.len(); info!(num_splits, "processing leaf search request (per-split)"); - // Process each split in parallel. The SearchPermitProvider inside - // SearcherContext gates concurrency based on memory budget. - let mut split_search_futures: Vec> = - Vec::with_capacity(all_splits.len()); + // Process each split in parallel using a JoinSet. The SearchPermitProvider + // inside SearcherContext gates concurrency based on memory budget. + let mut split_search_joinset: tokio::task::JoinSet<(String, Result<_, String>)> = + tokio::task::JoinSet::new(); for (leaf_req_idx, split) in all_splits { + let split_id = split.split_id.clone(); 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(), @@ -89,26 +92,42 @@ pub async fn handle_leaf_search( let searcher_context = ctx.searcher_context.clone(); let storage_resolver = ctx.storage_resolver.clone(); - split_search_futures.push(tokio::task::spawn(multi_index_leaf_search( - searcher_context, - single_split_request, - storage_resolver, - ))); + split_search_joinset.spawn(async move { + let result = + multi_index_leaf_search(searcher_context, single_split_request, storage_resolver) + .await + .map_err(|err| format!("{err}")); + (split_id, result) + }); } - // Collect results, preserving split order. - let mut responses: Vec = Vec::with_capacity(num_splits); - for split_search_fut in split_search_futures { - match split_search_fut.await { - Ok(Ok(response)) => responses.push(response), - Ok(Err(e)) => { - return Err(LambdaError::Internal(format!("leaf search failed: {e}"))); + // Collect results. Order is irrelevant: each result is tagged with its split_id. + let mut split_results: Vec = Vec::with_capacity(num_splits); + let mut num_successes: usize = 0; + let mut num_failures: usize = 0; + while let Some(join_result) = split_search_joinset.join_next().await { + match join_result { + Ok((split_id, Ok(response))) => { + num_successes += 1; + split_results.push(LambdaSingleSplitResult { + split_id, + outcome: Some(Outcome::Response(response)), + }); + } + Ok((split_id, Err(error_msg))) => { + num_failures += 1; + warn!(split_id = %split_id, error = %error_msg, "split search failed"); + split_results.push(LambdaSingleSplitResult { + split_id, + outcome: Some(Outcome::Error(error_msg)), + }); } Err(join_error) if join_error.is_cancelled() => { warn!("search task was cancelled"); return Err(LambdaError::Cancelled); } Err(join_error) => { + // Panics lose the captured split_id, so we fail the entire invocation. error!(error = %join_error, "search task panicked"); return Err(LambdaError::Internal(format!( "search task panicked: {join_error}" @@ -116,16 +135,14 @@ pub async fn handle_leaf_search( } } } - info!( - num_responses = responses.len(), - "leaf search completed (per-split)" + num_successes, + num_failures, "leaf search completed (per-split)" ); - // Serialize as LeafSearchResponses wrapper - let wrapper = LeafSearchResponses { responses }; + let wrapper = LambdaSearchResponses { split_results }; let response_bytes = wrapper.encode_to_vec(); let payload = BASE64_STANDARD.encode(&response_bytes); - Ok(LeafSearchResponsePayload { payload }) + Ok(LambdaSearchResponsePayload { payload }) } diff --git a/quickwit/quickwit-lambda-server/src/lib.rs b/quickwit/quickwit-lambda-server/src/lib.rs index 78f882d995f..da2519b3b33 100644 --- a/quickwit/quickwit-lambda-server/src/lib.rs +++ b/quickwit/quickwit-lambda-server/src/lib.rs @@ -23,4 +23,4 @@ mod handler; pub use context::LambdaSearcherContext; pub use error::{LambdaError, LambdaResult}; -pub use handler::{LeafSearchRequestPayload, LeafSearchResponsePayload, handle_leaf_search}; +pub use handler::{LambdaSearchRequestPayload, LambdaSearchResponsePayload, handle_leaf_search}; diff --git a/quickwit/quickwit-proto/protos/quickwit/search.proto b/quickwit/quickwit-proto/protos/quickwit/search.proto index d6a79ddace5..ae3442fe1aa 100644 --- a/quickwit/quickwit-proto/protos/quickwit/search.proto +++ b/quickwit/quickwit-proto/protos/quickwit/search.proto @@ -333,8 +333,7 @@ message SplitSearchError { bool retryable_error = 3; } -/// A LeafSearchRequest can span multiple indices. -/// +// A LeafSearchRequest can span multiple indices. message LeafSearchRequest { // Search request. This is a perfect copy of the original search request // that was sent to root apart from the start_offset, max_hits params and index_id_patterns. @@ -361,7 +360,7 @@ message ResourceStats { uint64 cpu_microsecs = 5; } -/// LeafRequestRef references data in LeafSearchRequest to deduplicate data. +// LeafRequestRef references data in LeafSearchRequest to deduplicate data. message LeafRequestRef { // The ordinal of the doc_mapper in `LeafSearchRequest.doc_mappers` uint32 doc_mapper_ord = 1; @@ -497,10 +496,23 @@ 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; +// The result of searching a single split in a Lambda invocation. +// Each result is tagged with its split_id so that ordering is irrelevant. +message LambdaSingleSplitResult { + // The split that was searched. + string split_id = 1; + oneof outcome { + // On success, the leaf search response for this split. + LeafSearchResponse response = 2; + // On failure, the error message. + string error = 3; + } +} + +// Wrapper for per-split results from a Lambda invocation. +message LambdaSearchResponses { + reserved 1; // was: repeated LeafSearchResponse responses + repeated LambdaSingleSplitResult split_results = 2; } message SnippetRequest { diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs index 997b93b6869..1e933055cd3 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.search.rs @@ -256,8 +256,7 @@ pub struct SplitSearchError { #[prost(bool, tag = "3")] pub retryable_error: bool, } -/// / A LeafSearchRequest can span multiple indices. -/// / +/// A LeafSearchRequest can span multiple indices. #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct LeafSearchRequest { @@ -292,7 +291,7 @@ pub struct ResourceStats { #[prost(uint64, tag = "5")] pub cpu_microsecs: u64, } -/// / LeafRequestRef references data in LeafSearchRequest to deduplicate data. +/// LeafRequestRef references data in LeafSearchRequest to deduplicate data. #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, ::prost::Message)] pub struct LeafRequestRef { @@ -460,13 +459,37 @@ 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. +/// The result of searching a single split in a Lambda invocation. +/// Each result is tagged with its split_id so that ordering is irrelevant. #[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, +pub struct LambdaSingleSplitResult { + /// The split that was searched. + #[prost(string, tag = "1")] + pub split_id: ::prost::alloc::string::String, + #[prost(oneof = "lambda_single_split_result::Outcome", tags = "2, 3")] + pub outcome: ::core::option::Option, +} +/// Nested message and enum types in `LambdaSingleSplitResult`. +pub mod lambda_single_split_result { + #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] + #[serde(rename_all = "snake_case")] + #[derive(Clone, PartialEq, ::prost::Oneof)] + pub enum Outcome { + /// On success, the leaf search response for this split. + #[prost(message, tag = "2")] + Response(super::LeafSearchResponse), + /// On failure, the error message. + #[prost(string, tag = "3")] + Error(::prost::alloc::string::String), + } +} +/// Wrapper for per-split results from a Lambda invocation. +#[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] +#[derive(Clone, PartialEq, ::prost::Message)] +pub struct LambdaSearchResponses { + #[prost(message, repeated, tag = "2")] + pub split_results: ::prost::alloc::vec::Vec, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[derive(Clone, PartialEq, Eq, Hash, ::prost::Message)] diff --git a/quickwit/quickwit-search/src/invoker.rs b/quickwit/quickwit-search/src/invoker.rs index f160d88019f..8a5ad183b63 100644 --- a/quickwit/quickwit-search/src/invoker.rs +++ b/quickwit/quickwit-search/src/invoker.rs @@ -15,7 +15,7 @@ //! Trait for invoking remote serverless functions for leaf search. use async_trait::async_trait; -use quickwit_proto::search::{LeafSearchRequest, LeafSearchResponse}; +use quickwit_proto::search::{LambdaSingleSplitResult, LeafSearchRequest}; use crate::SearchError; @@ -27,9 +27,12 @@ use crate::SearchError; pub trait LambdaLeafSearchInvoker: Send + Sync + 'static { /// Invoke the remote function with a LeafSearchRequest. /// - /// Returns one `LeafSearchResponse` per split in the request. + /// Returns one `LambdaSingleSplitResult` per split in the request. + /// Each result is tagged with its split_id so ordering is irrelevant. + /// Individual split failures are reported per-split; the outer `Result` + /// only represents transport-level errors. async fn invoke_leaf_search( &self, request: LeafSearchRequest, - ) -> Result, SearchError>; + ) -> Result, SearchError>; } diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 9bb32f7c2c3..92ff467c0be 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -27,6 +27,7 @@ use quickwit_common::pretty::PrettySample; use quickwit_common::uri::Uri; use quickwit_directories::{CachingDirectory, HotDirectory, StorageDirectory}; use quickwit_doc_mapper::{Automaton, DocMapper, FastFieldWarmupInfo, TermRange, WarmupInfo}; +use quickwit_proto::search::lambda_single_split_result::Outcome; use quickwit_proto::search::{ CountHits, LeafSearchRequest, LeafSearchResponse, PartialHit, ResourceStats, SearchRequest, SortOrder, SortValue, SplitIdAndFooterOffsets, SplitSearchError, @@ -1397,14 +1398,17 @@ async fn run_offloaded_search_tasks( request: &SearchRequest, doc_mapper: &DocMapper, index_uri: Uri, - splits: Vec, + splits_with_requests: Vec<(SplitIdAndFooterOffsets, SearchRequest)>, incremental_merge_collector: &Mutex, ) -> Result<(), SearchError> { - if splits.is_empty() { + if splits_with_requests.is_empty() { return Ok(()); } - info!(num_offloaded_splits = splits.len(), "offloading to lambda"); + info!( + num_offloaded_splits = splits_with_requests.len(), + "offloading to lambda" + ); let lambda_invoker = searcher_context.lambda_invoker.as_ref().expect( "did not receive enough permit futures despite not having any lambda invoker to offload to", @@ -1414,6 +1418,18 @@ async fn run_offloaded_search_tasks( let doc_mapper_str = serde_json::to_string(doc_mapper) .map_err(|err| SearchError::Internal(format!("failed to serialize doc mapper: {err}")))?; + // Build a lookup so we can match lambda results (tagged by split_id) back to the + // split metadata and per-split SearchRequest needed for caching. + let mut split_lookup: HashMap = + HashMap::with_capacity(splits_with_requests.len()); + let splits: Vec = splits_with_requests + .into_iter() + .map(|(split, search_req)| { + split_lookup.insert(split.split_id.clone(), (split.clone(), search_req)); + split + }) + .collect(); + let batches: Vec> = greedy_batch_split( splits, |split| split.num_docs, @@ -1452,21 +1468,51 @@ async fn run_offloaded_search_tasks( error!("lambda join error"); return Err(SearchError::Internal("lambda join error".to_string())); }; - let mut locked = incremental_merge_collector.lock().unwrap(); match result { - Ok(per_split_responses) => { - for response in per_split_responses { - if let Err(err) = locked.add_result(response) { - error!(error = %err, "failed to add lambda result to collector"); + Ok(split_results) => { + let mut locked = incremental_merge_collector.lock().unwrap(); + for split_result in split_results { + match split_result.outcome { + Some(Outcome::Response(response)) => { + if let Some((split_info, search_req)) = + split_lookup.remove(&split_result.split_id) + { + searcher_context.leaf_search_cache.put( + split_info, + search_req, + response.clone(), + ); + } + if let Err(err) = locked.add_result(response) { + error!(error = %err, "failed to add lambda result to collector"); + } + } + Some(Outcome::Error(error_msg)) => { + locked.add_failed_split(SplitSearchError { + split_id: split_result.split_id, + error: format!("lambda split error: {error_msg}"), + retryable_error: true, + }); + } + None => { + locked.add_failed_split(SplitSearchError { + split_id: split_result.split_id, + error: "lambda returned empty outcome".to_string(), + retryable_error: true, + }); + } } } } Err(err) => { + // Transport-level failure: the Lambda invocation itself failed. + // Mark all splits in this batch as failed. error!( error = %err, num_splits = batch_split_ids.len(), "lambda invocation failed for batch" ); + let mut locked = incremental_merge_collector.lock().unwrap(); for split_id in batch_split_ids { locked.add_failed_split(SplitSearchError { split_id, @@ -1491,7 +1537,9 @@ struct ScheduleSearchTaskResult { // The search permit futures associated to each local_search_task are // guaranteed to resolve in order. local_search_tasks: Vec, - offloaded_search_tasks: Vec, + // The per-split SearchRequest (already rewritten by `rewrite_request()`) is preserved + // so that lambda results can be cached with the correct cache key in `leaf_search_cache`. + offloaded_search_tasks: Vec<(SplitIdAndFooterOffsets, SearchRequest)>, } /// Schedule search tasks, either: @@ -1527,10 +1575,8 @@ async fn schedule_search_tasks( .get_permits_with_offload(permit_sizes, offload_threshold) .await; - let splits_to_run_on_lambda: Vec = splits - .drain(search_permit_futures.len()..) - .map(|(split, _req)| split) - .collect(); + let splits_to_run_on_lambda: Vec<(SplitIdAndFooterOffsets, SearchRequest)> = + splits.drain(search_permit_futures.len()..).collect(); let splits_to_run_locally: Vec = splits .into_iter() @@ -1872,6 +1918,7 @@ mod tests { use bytes::BufMut; use quickwit_config::{LambdaConfig, SearcherConfig}; use quickwit_directories::write_hotcache; + use quickwit_proto::search::LambdaSingleSplitResult; use rand::Rng; use tantivy::TantivyDocument; use tantivy::directory::RamDirectory; @@ -2507,7 +2554,7 @@ mod tests { async fn invoke_leaf_search( &self, _req: LeafSearchRequest, - ) -> Result, SearchError> { + ) -> Result, SearchError> { todo!() } } @@ -2527,7 +2574,7 @@ mod tests { for (idx, task) in result.local_search_tasks.iter().enumerate() { assert_eq!(task.split.split_id, format!("split_{idx}")); } - for (idx, split) in result.offloaded_search_tasks.iter().enumerate() { + for (idx, (split, _req)) in result.offloaded_search_tasks.iter().enumerate() { assert_eq!(split.split_id, format!("split_{}", idx + 3)); } } From ad71de4c6ff3e48e1cc4033e57f223fcb78ec3cd Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Thu, 19 Feb 2026 10:30:09 +0100 Subject: [PATCH 4/6] CR comments --- quickwit/Cargo.lock | 1 + quickwit/quickwit-lambda-server/Cargo.toml | 1 + .../quickwit-lambda-server/src/handler.rs | 146 ++++++++++++------ .../src/query_ast/range_query.rs | 12 +- quickwit/quickwit-search/src/leaf.rs | 55 ++++--- quickwit/quickwit-search/src/service.rs | 3 +- 6 files changed, 132 insertions(+), 86 deletions(-) diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index d5e1d4a9d84..7836dd5a74b 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -7132,6 +7132,7 @@ dependencies = [ "prost 0.14.1", "quickwit-common", "quickwit-config", + "quickwit-doc-mapper", "quickwit-proto", "quickwit-search", "quickwit-storage", diff --git a/quickwit/quickwit-lambda-server/Cargo.toml b/quickwit/quickwit-lambda-server/Cargo.toml index 1bfebfcc655..e362c329e1f 100644 --- a/quickwit/quickwit-lambda-server/Cargo.toml +++ b/quickwit/quickwit-lambda-server/Cargo.toml @@ -32,6 +32,7 @@ openssl = { workspace = true, optional = 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 } diff --git a/quickwit/quickwit-lambda-server/src/handler.rs b/quickwit/quickwit-lambda-server/src/handler.rs index 5312f8337bd..e0467af21ef 100644 --- a/quickwit/quickwit-lambda-server/src/handler.rs +++ b/quickwit/quickwit-lambda-server/src/handler.rs @@ -12,13 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::str::FromStr; +use std::sync::Arc; + use base64::prelude::*; use prost::Message; +use quickwit_common::uri::Uri; +use quickwit_doc_mapper::DocMapper; use quickwit_proto::search::lambda_single_split_result::Outcome; use quickwit_proto::search::{ - LambdaSearchResponses, LambdaSingleSplitResult, LeafSearchRequest, SplitIdAndFooterOffsets, + LambdaSearchResponses, LambdaSingleSplitResult, LeafRequestRef, LeafSearchRequest, + SearchRequest, }; -use quickwit_search::leaf::multi_index_leaf_search; +use quickwit_search::leaf::single_doc_mapping_leaf_search; +use quickwit_storage::Storage; use serde::{Deserialize, Serialize}; use tracing::{error, info, instrument, warn}; @@ -50,57 +57,106 @@ pub async fn handle_leaf_search( ctx: &LambdaSearcherContext, ) -> LambdaResult { // Decode base64 payload - let request_bytes = BASE64_STANDARD + let request_bytes: Vec = BASE64_STANDARD .decode(&event.payload) - .map_err(|e| LambdaError::Serialization(format!("base64 decode error: {}", e)))?; + .map_err(|err| LambdaError::Serialization(format!("base64 decode error: {}", err)))?; // Deserialize LeafSearchRequest let leaf_search_request = LeafSearchRequest::decode(&request_bytes[..])?; - let all_splits: Vec<(usize, SplitIdAndFooterOffsets)> = - leaf_search_request - .leaf_requests - .iter() - .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(); + // Unpack the shared fields once instead of cloning per split. + let search_request: Arc = leaf_search_request + .search_request + .ok_or_else(|| LambdaError::Internal("no search request".to_string()))? + .into(); - let num_splits = all_splits.len(); - info!(num_splits, "processing leaf search request (per-split)"); + let doc_mappers: Vec> = leaf_search_request + .doc_mappers + .iter() + .map(String::as_str) + .map(serde_json::from_str::>) + .collect::, _>>() + .map_err(|err| { + LambdaError::Internal(format!("failed to deserialize doc mapper: `{err}`")) + })?; + + // Resolve storage for every index URI upfront. + let mut storages: Vec> = + Vec::with_capacity(leaf_search_request.index_uris.len()); + for uri_str in &leaf_search_request.index_uris { + let uri = Uri::from_str(uri_str) + .map_err(|err| LambdaError::Internal(format!("invalid index uri: {err}")))?; + let storage = + ctx.storage_resolver.resolve(&uri).await.map_err(|err| { + LambdaError::Internal(format!("failed to resolve storage: {err}")) + })?; + storages.push(storage); + } + + let split_results: Vec = lambda_leaf_search( + search_request, + leaf_search_request.leaf_requests, + &doc_mappers[..], + &storages[..], + ctx, + ) + .await?; + let wrapper = LambdaSearchResponses { split_results }; + let response_bytes = wrapper.encode_to_vec(); + let payload = BASE64_STANDARD.encode(&response_bytes); + + Ok(LambdaSearchResponsePayload { payload }) +} - // Process each split in parallel using a JoinSet. The SearchPermitProvider - // inside SearcherContext gates concurrency based on memory budget. +/// Lambda leaf search returns individual split results. +async fn lambda_leaf_search( + search_request: Arc, + leaf_req_ref: Vec, + doc_mappers: &[Arc], + storages: &[Arc], + ctx: &LambdaSearcherContext, +) -> LambdaResult> { + // Flatten leaf_requests into per-split tasks using pre-resolved Arc references. let mut split_search_joinset: tokio::task::JoinSet<(String, Result<_, String>)> = tokio::task::JoinSet::new(); - for (leaf_req_idx, split) in all_splits { - let split_id = split.split_id.clone(); - 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(); - split_search_joinset.spawn(async move { - let result = - multi_index_leaf_search(searcher_context, single_split_request, storage_resolver) - .await - .map_err(|err| format!("{err}")); - (split_id, result) - }); + + for leaf_req in leaf_req_ref { + let doc_mapper = doc_mappers + .get(leaf_req.doc_mapper_ord as usize) + .ok_or_else(|| { + LambdaError::Internal(format!( + "doc_mapper_ord out of bounds: {}", + leaf_req.doc_mapper_ord + )) + })? + .clone(); + let storage = storages[leaf_req.index_uri_ord as usize].clone(); + + for split_id_and_footer_offsets in leaf_req.split_offsets { + let split_id = split_id_and_footer_offsets.split_id.clone(); + let searcher_context = ctx.searcher_context.clone(); + let search_request = search_request.clone(); + let doc_mapper = doc_mapper.clone(); + let storage = storage.clone(); + let split = split_id_and_footer_offsets.clone(); + split_search_joinset.spawn(async move { + let result = single_doc_mapping_leaf_search( + searcher_context, + search_request, + storage, + vec![split], + doc_mapper, + ) + .await + .map_err(|err| format!("{err}")); + (split_id, result) + }); + } } + let num_splits = split_search_joinset.len(); + info!(num_splits, "processing leaf search request (per-split)"); + // Collect results. Order is irrelevant: each result is tagged with its split_id. let mut split_results: Vec = Vec::with_capacity(num_splits); let mut num_successes: usize = 0; @@ -140,9 +196,5 @@ pub async fn handle_leaf_search( num_failures, "leaf search completed (per-split)" ); - let wrapper = LambdaSearchResponses { split_results }; - let response_bytes = wrapper.encode_to_vec(); - let payload = BASE64_STANDARD.encode(&response_bytes); - - Ok(LambdaSearchResponsePayload { payload }) + Ok(split_results) } diff --git a/quickwit/quickwit-query/src/query_ast/range_query.rs b/quickwit/quickwit-query/src/query_ast/range_query.rs index 3f5c4471ae4..346f02feed3 100644 --- a/quickwit/quickwit-query/src/query_ast/range_query.rs +++ b/quickwit/quickwit-query/src/query_ast/range_query.rs @@ -181,8 +181,8 @@ impl BuildTantivyAst for RangeQuery { convert_bounds(&self.lower_bound, &self.upper_bound, field_entry.name())?; let truncate_datetime = |date: &DateTime| date.truncate(date_options.get_precision()); - let lower_bound = map_bound(&lower_bound, truncate_datetime); - let upper_bound = map_bound(&upper_bound, truncate_datetime); + let lower_bound = lower_bound.as_ref().map(truncate_datetime); + let upper_bound = upper_bound.as_ref().map(truncate_datetime); FastFieldRangeQuery::new( lower_bound.map(|val| Term::from_field_date(field, val)), upper_bound.map(|val| Term::from_field_date(field, val)), @@ -277,14 +277,6 @@ impl BuildTantivyAst for RangeQuery { } } -fn map_bound(bound: &Bound, transform: impl Fn(&TFrom) -> TTo) -> Bound { - match bound { - Bound::Excluded(from_val) => Bound::Excluded(transform(from_val)), - Bound::Included(from_val) => Bound::Included(transform(from_val)), - Bound::Unbounded => Bound::Unbounded, - } -} - #[cfg(test)] mod tests { use std::ops::Bound; diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 92ff467c0be..97d5c1a9869 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -499,6 +499,17 @@ async fn leaf_search_single_split( let mut leaf_search_state_guard = SplitSearchStateGuard::new(ctx.split_outcome_counters.clone()); + // We already checked if the result was already in the partial result cache, + // but it's not a bad idea to check again. + if let Some(cached_answer) = ctx + .searcher_context + .leaf_search_cache + .get(split.clone(), search_request.clone()) + { + leaf_search_state_guard.set_state(SplitSearchState::CacheHit); + return Ok(Some(cached_answer)); + } + let query_ast: QueryAst = serde_json::from_str(search_request.query_ast.as_str()) .map_err(|err| SearchError::InvalidQuery(err.to_string()))?; @@ -721,17 +732,6 @@ fn visit_aggregation_mut( modified_something } -/// 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 { - Unbounded => Unbounded, - Included(x) => Included(f(x)), - Excluded(x) => Excluded(f(x)), - } -} - // returns the max of left and right, that isn't unbounded. Useful for making // the intersection of lower bound of ranges fn max_bound(left: Bound, right: Bound) -> Bound { @@ -864,12 +864,8 @@ fn remove_redundant_timestamp_range( if final_start_timestamp != Bound::Unbounded || final_end_timestamp != Bound::Unbounded { let range = RangeQuery { field: timestamp_field.to_string(), - lower_bound: map_bound(final_start_timestamp, |bound| { - bound.into_timestamp_nanos().into() - }), - upper_bound: map_bound(final_end_timestamp, |bound| { - bound.into_timestamp_nanos().into() - }), + lower_bound: final_start_timestamp.map(|bound| bound.into_timestamp_nanos().into()), + upper_bound: final_end_timestamp.map(|bound| bound.into_timestamp_nanos().into()), }; new_ast = if let QueryAst::Bool(mut bool_query) = new_ast { if bool_query.must.is_empty() @@ -1390,12 +1386,12 @@ fn disable_search_request_hits(search_request: &mut SearchRequest) { } /// Searches multiple splits for a specific index and a single doc mapping -/// Offloads splits to Lambda invocations, distributing them accross batches +/// Offloads splits to Lambda invocations, distributing them across batches /// balanced by document count. Each batch is invoked independently; a failure /// in one batch does not affect others. async fn run_offloaded_search_tasks( searcher_context: &SearcherContext, - request: &SearchRequest, + search_request: &SearchRequest, doc_mapper: &DocMapper, index_uri: Uri, splits_with_requests: Vec<(SplitIdAndFooterOffsets, SearchRequest)>, @@ -1436,16 +1432,14 @@ async fn run_offloaded_search_tasks( lambda_config.max_splits_per_invocation, ); - 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_joinset = JoinSet::new(); for batch in batches { let batch_split_ids: Vec = batch.iter().map(|split| split.split_id.clone()).collect(); let leaf_request = LeafSearchRequest { - search_request: Some(search_request_for_leaf.clone()), + // Note this is not the split-specific rewritten request, we ship the main request, + // and the leaf will apply the split specific rewrite on its own. + search_request: Some(search_request.clone()), doc_mappers: vec![doc_mapper_str.clone()], index_uris: vec![index_uri.as_str().to_string()], //< careful here. Calling to_string() directly would return a redacted uri. leaf_requests: vec![quickwit_proto::search::LeafRequestRef { @@ -1474,12 +1468,13 @@ async fn run_offloaded_search_tasks( for split_result in split_results { match split_result.outcome { Some(Outcome::Response(response)) => { - if let Some((split_info, search_req)) = + if let Some((split_info, single_split_search_req)) = split_lookup.remove(&split_result.split_id) { + // We use the single_split_search_req to perform the search searcher_context.leaf_search_cache.put( split_info, - search_req, + single_split_search_req, response.clone(), ); } @@ -1616,8 +1611,12 @@ pub async fn single_doc_mapping_leaf_search( CanSplitDoBetter::from_request(&request, doc_mapper.timestamp_field_name()); let mut split_with_req: Vec<(SplitIdAndFooterOffsets, SearchRequest)> = split_filter.optimize(&request, splits)?; - for (split, search_request) in &mut split_with_req { - rewrite_request(search_request, split, doc_mapper.timestamp_field_name()); + for (split, single_split_search_request) in &mut split_with_req { + rewrite_request( + single_split_search_request, + split, + doc_mapper.timestamp_field_name(), + ); } let split_filter_arc: Arc> = Arc::new(RwLock::new(split_filter)); diff --git a/quickwit/quickwit-search/src/service.rs b/quickwit/quickwit-search/src/service.rs index a82edf2d9fc..5ee74c16556 100644 --- a/quickwit/quickwit-search/src/service.rs +++ b/quickwit/quickwit-search/src/service.rs @@ -153,7 +153,8 @@ impl SearchServiceImpl { } } -pub fn deserialize_doc_mapper(doc_mapper_str: &str) -> crate::Result> { +/// Deserializes a JSON-encoded doc mapper string into an `Arc`. +pub(crate) fn deserialize_doc_mapper(doc_mapper_str: &str) -> crate::Result> { let doc_mapper = serde_json::from_str::>(doc_mapper_str).map_err(|err| { SearchError::Internal(format!("failed to deserialize doc mapper: `{err}`")) })?; From c7be288b285bf1adb658304f910a545883e8c947 Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Thu, 19 Feb 2026 12:40:48 +0100 Subject: [PATCH 5/6] CR comment: using binary heap in greedy_batch_split --- quickwit/quickwit-search/src/leaf.rs | 35 +++++++++++++++++----------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 97d5c1a9869..9bf1ed862e5 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -12,7 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet}; +use std::cmp::Reverse; +use std::collections::binary_heap::PeekMut; +use std::collections::{BinaryHeap, HashMap, HashSet}; use std::num::NonZeroUsize; use std::ops::Bound; use std::path::PathBuf; @@ -87,23 +89,28 @@ fn greedy_batch_split( } weighted_items.sort_unstable_by_key(|(weight, _)| std::cmp::Reverse(*weight)); - // Invariant: batch_weights[i] is the sum of weights in batches[i] let mut batches: Vec> = std::iter::repeat_with(Vec::new).take(num_batches).collect(); - let mut batch_weights: Vec = vec![0; num_batches]; + + // Min-heap of (weight, item_count, batch_index). + // Reverse turns BinaryHeap into a min-heap. + // Ties break naturally: lighter weight → fewer items → lower index. + let mut heap: BinaryHeap> = + BinaryHeap::with_capacity(num_batches); + for batch_idx in 0..num_batches { + heap.push(Reverse((0, 0, batch_idx))); + } // Greedily assign each item to the lightest batch. - // Ties are broken by count to help balance item counts when weights are equal. + // Full batches are removed via PeekMut::pop(). for (weight, item) in weighted_items { - let lightest_batch_idx = batch_weights - .iter() - .zip(batches.iter()) - .enumerate() - .filter(|(_, (_, batch))| batch.len() < max_items_per_batch) - .min_by_key(|(_, (batch_weight, batch))| (**batch_weight, batch.len())) - .map(|(idx, _)| idx) - .unwrap(); - batch_weights[lightest_batch_idx] += weight; - batches[lightest_batch_idx].push(item); + let mut top = heap.peek_mut().unwrap(); + let Reverse((ref mut batch_weight, ref mut batch_count, batch_idx)) = *top; + batches[batch_idx].push(item); + *batch_weight += weight; + *batch_count += 1; + if *batch_count >= max_items_per_batch { + PeekMut::pop(top); + } } batches From 1127e128b3711e5041f02da49814ae3607e0b88e Mon Sep 17 00:00:00 2001 From: fulmicoton Date: Thu, 19 Feb 2026 13:01:44 +0100 Subject: [PATCH 6/6] Adding assert on search permit provider unit test --- docs/configuration/lambda-config.md | 6 +-- quickwit/quickwit-lambda-client/src/deploy.rs | 6 +-- quickwit/quickwit-search/src/leaf.rs | 3 +- .../src/search_permit_provider.rs | 50 ++++++++++--------- 4 files changed, 34 insertions(+), 31 deletions(-) diff --git a/docs/configuration/lambda-config.md b/docs/configuration/lambda-config.md index 5b2184c32fd..4f522da4b35 100644 --- a/docs/configuration/lambda-config.md +++ b/docs/configuration/lambda-config.md @@ -76,7 +76,7 @@ This is the recommended approach as it ensures the Lambda function always matche You can deploy the Lambda function manually without `auto_deploy`: 1. Download the Lambda zip from [GitHub releases](https://github.com/quickwit-oss/quickwit/releases) 2. Create or update the Lambda function using AWS CLI, Terraform, or the AWS Console -3. Publish a version with description format `quickwit:{version}-{sha1}` (e.g., `quickwit:0_8_0-fa752891`) +3. Publish a version with description format `quickwit_{version}_{sha256}_{timeout}_{deploy_config}"` (e.g., `quickwit_0_8_0_fa940f44_5120_60s_6c3b2`) The description must match the format Quickwit expects, or it won't find the function version. @@ -195,8 +195,8 @@ No additional configuration is needed on the Quickwit side. ## Versioning Quickwit uses content-based versioning for Lambda: -- A SHA1 hash of the Lambda binary is computed at build time -- This hash is embedded in the Lambda function description as `quickwit:{version}-{sha1_short}` +- A SHA256 hash of the Lambda binary is computed at build time +- This hash is embedded in the Lambda function description as `quickwit:{version}-{sha256_short}` - When Quickwit starts, it searches for a version matching this description - Different Quickwit builds with the same Lambda binary share the same Lambda version - Updating the Lambda binary automatically triggers a new deployment diff --git a/quickwit/quickwit-lambda-client/src/deploy.rs b/quickwit/quickwit-lambda-client/src/deploy.rs index 4fc1b637596..e189e4a7ec0 100644 --- a/quickwit/quickwit-lambda-client/src/deploy.rs +++ b/quickwit/quickwit-lambda-client/src/deploy.rs @@ -46,7 +46,7 @@ use crate::invoker::create_lambda_invoker_for_version; 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:"; +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; @@ -59,7 +59,7 @@ fn lambda_qualifier() -> &'static str { LAMBDA_QUALIFIER .get_or_init(|| { format!( - "{}-{}", + "{}_{}", env!("CARGO_PKG_VERSION").replace('.', "_"), env!("LAMBDA_BINARY_HASH") ) @@ -70,7 +70,7 @@ fn lambda_qualifier() -> &'static str { /// Returns the version description for our qualifier. /// /// We also pass the deploy config, as we want the function to be redeployed -/// if the deployed is changed. +/// if the deploy config is changed. fn version_description(deploy_config_opt: Option<&LambdaDeployConfig>) -> String { if let Some(deploy_config) = deploy_config_opt { let memory_size_mib = deploy_config.memory_size.as_mib() as u64; diff --git a/quickwit/quickwit-search/src/leaf.rs b/quickwit/quickwit-search/src/leaf.rs index 9bf1ed862e5..37f99a36241 100644 --- a/quickwit/quickwit-search/src/leaf.rs +++ b/quickwit/quickwit-search/src/leaf.rs @@ -94,8 +94,7 @@ fn greedy_batch_split( // Min-heap of (weight, item_count, batch_index). // Reverse turns BinaryHeap into a min-heap. // Ties break naturally: lighter weight → fewer items → lower index. - let mut heap: BinaryHeap> = - BinaryHeap::with_capacity(num_batches); + let mut heap: BinaryHeap> = BinaryHeap::with_capacity(num_batches); for batch_idx in 0..num_batches { heap.push(Reverse((0, 0, batch_idx))); } diff --git a/quickwit/quickwit-search/src/search_permit_provider.rs b/quickwit/quickwit-search/src/search_permit_provider.rs index cc59958b0d1..4cd5e99918f 100644 --- a/quickwit/quickwit-search/src/search_permit_provider.rs +++ b/quickwit/quickwit-search/src/search_permit_provider.rs @@ -16,13 +16,12 @@ use std::collections::BinaryHeap; use std::collections::binary_heap::PeekMut; use std::future::Future; use std::pin::Pin; +use std::sync::Arc; use std::task::{Context, Poll}; use bytesize::ByteSize; use quickwit_common::metrics::GaugeGuard; use quickwit_proto::search::SplitIdAndFooterOffsets; -#[cfg(test)] -use tokio::sync::watch; use tokio::sync::{mpsc, oneshot}; /// Distributor of permits to perform split search operation. @@ -35,8 +34,8 @@ use tokio::sync::{mpsc, oneshot}; #[derive(Clone)] pub struct SearchPermitProvider { message_sender: mpsc::UnboundedSender, - #[cfg(test)] - actor_stopped: watch::Receiver, + #[allow(dead_code)] + actor_join_handle: Arc>, } pub enum SearchPermitMessage { @@ -88,8 +87,6 @@ pub fn compute_initial_memory_allocation( impl SearchPermitProvider { pub fn new(num_download_slots: usize, memory_budget: ByteSize) -> Self { let (message_sender, message_receiver) = mpsc::unbounded_channel(); - #[cfg(test)] - let (state_sender, state_receiver) = watch::channel(false); let actor = SearchPermitActor { msg_receiver: message_receiver, msg_sender: message_sender.downgrade(), @@ -97,17 +94,25 @@ impl SearchPermitProvider { total_memory_budget: memory_budget.as_u64(), permits_requests: BinaryHeap::new(), total_memory_allocated: 0u64, - #[cfg(test)] - stopped: state_sender, }; - tokio::spawn(actor.run()); + let actor_join_handle = Arc::new(tokio::spawn(actor.run())); Self { message_sender, - #[cfg(test)] - actor_stopped: state_receiver, + actor_join_handle, } } + #[cfg(test)] + async fn stop_and_unwrap(self) -> SearchPermitActor { + let SearchPermitProvider { + message_sender, + actor_join_handle, + .. + } = self; + drop(message_sender); + Arc::into_inner(actor_join_handle).unwrap().await.unwrap() + } + /// Returns permits for local splits /// /// The returned futures are guaranteed to resolve in order. @@ -158,8 +163,6 @@ struct SearchPermitActor { total_memory_budget: u64, total_memory_allocated: u64, permits_requests: BinaryHeap, - #[cfg(test)] - stopped: watch::Sender, } struct SingleSplitPermitRequest { @@ -237,13 +240,12 @@ impl LeafPermitRequest { } impl SearchPermitActor { - async fn run(mut self) { + async fn run(mut self) -> Self { // Stops when the last clone of SearchPermitProvider is dropped. while let Some(msg) = self.msg_receiver.recv().await { self.handle_message(msg); } - #[cfg(test)] - self.stopped.send(true).ok(); + self } fn handle_message(&mut self, msg: SearchPermitMessage) { @@ -562,7 +564,7 @@ mod tests { drop(permit_fut1); let permit = permit_fut2.await; assert_eq!(permit.memory_allocation, ByteSize::mb(10).as_u64()); - assert_eq!(*permit_provider.actor_stopped.borrow(), false); + assert!(!permit_provider.actor_join_handle.is_finished()); let _permit_fut3 = permit_provider .get_permits(vec![ByteSize::mb(10)]) @@ -570,12 +572,12 @@ mod tests { .into_iter() .next() .unwrap(); - let mut actor_stopped = permit_provider.actor_stopped.clone(); - drop(permit_provider); - { - actor_stopped.changed().await.unwrap(); - assert!(*actor_stopped.borrow()); - } + let SearchPermitProvider { + message_sender, + actor_join_handle, + } = permit_provider; + drop(message_sender); + Arc::into_inner(actor_join_handle).unwrap().await.unwrap(); } /// Tries to wait for a permit @@ -631,6 +633,8 @@ mod tests { .get_permits_with_offload(vec![ByteSize::mb(1); 5], 0) .await; assert!(permits.is_empty()); + let permit_actor = permit_provider.stop_and_unwrap().await; + assert!(permit_actor.permits_requests.is_empty()); } #[tokio::test]