diff --git a/config/tutorials/hdfs-logs/index-config-partitioned.yaml b/config/tutorials/hdfs-logs/index-config-partitioned.yaml index 5a40a9f6764..95c53d29423 100644 --- a/config/tutorials/hdfs-logs/index-config-partitioned.yaml +++ b/config/tutorials/hdfs-logs/index-config-partitioned.yaml @@ -43,5 +43,3 @@ indexing_settings: merge_factor: 10 max_merge_ops: 3 maturation_period: 48 hours - resources: - max_merge_write_throughput: 100mb diff --git a/docs/reference/cli.md b/docs/reference/cli.md index 651e162013d..e97e9effb59 100644 --- a/docs/reference/cli.md +++ b/docs/reference/cli.md @@ -182,39 +182,15 @@ quickwit index create --endpoint=http://127.0.0.1:7280 --index-config wikipedia_ ### index update +Updates an index using an index config file. `quickwit index update [args]` -#### index update search-settings - -Updates default search settings. -`quickwit index update search-settings [args]` *Synopsis* ```bash -quickwit index update search-settings +quickwit index update --index - --default-search-fields -``` - -*Options* - -| Option | Description | -|-----------------|-------------| -| `--index` | ID of the target index | -| `--default-search-fields` | List of fields that Quickwit will search into if the user query does not explicitly target a field. Space-separated list, e.g. "field1 field2". If no value is provided, existing defaults are removed and queries without target field will fail. | -#### index update retention-policy - -Configure or disable the retention policy. -`quickwit index update retention-policy [args]` - -*Synopsis* - -```bash -quickwit index update retention-policy - --index - [--period ] - [--schedule ] - [--disable] + --index-config ``` *Options* @@ -222,9 +198,7 @@ quickwit index update retention-policy | Option | Description | |-----------------|-------------| | `--index` | ID of the target index | -| `--period` | Duration after which splits are dropped. Expressed in a human-readable way (`1 day`, `2 hours`, `1 week`, ...) | -| `--schedule` | Frequency at which the retention policy is evaluated and applied. Expressed as a cron expression (0 0 * * * *) or human-readable form (hourly, daily, weekly, ...). | -| `--disable` | Disable the retention policy. Old indexed data will not be cleaned up anymore. | +| `--index-config` | Location of the index config file. | ### index clear Clears an index: deletes all splits and resets checkpoint. @@ -381,7 +355,7 @@ quickwit index ingest | `--batch-size-limit` | Size limit of each submitted document batch. | | `--wait` | Wait for all documents to be commited and available for search before exiting | | `--force` | Force a commit after the last document is sent, and wait for all documents to be committed and available for search before exiting | -| `--commit-timeout` | Timeout for ingest operations that require waiting for the final commit (`--wait` or `--force`). This is different from the `commit_timeout_secs` indexing setting which sets the maximum time before commiting splits after their creation. | +| `--commit-timeout` | Timeout for ingest operations that require waiting for the final commit (`--wait` or `--force`). This is different from the `commit_timeout_secs` indexing setting, which sets the maximum time before commiting splits after their creation. | *Examples* diff --git a/docs/reference/rest-api.md b/docs/reference/rest-api.md index d0606b74372..7835b1b1d8a 100644 --- a/docs/reference/rest-api.md +++ b/docs/reference/rest-api.md @@ -58,18 +58,18 @@ POST api/v1//search #### Parameters -| Variable | Type | Description | Default value | -|---------------------|------------|--------------------------------------------------------------------------------------------------------------------------------------------------------|----------------------------------------------------| -| `query` | `String` | Query text. See the [query language doc](query-language.md) (mandatory) | | -| `start_timestamp` | `i64` | If set, restrict search to documents with a `timestamp >= start_timestamp`, taking advantage of potential time pruning oportunities. The value must be in seconds. | | -| `end_timestamp` | `i64` | If set, restrict search to documents with a `timestamp < end_timestamp`, taking advantage of potential time pruning oportunities. The value must be in seconds. | | -| `start_offset` | `Integer` | Number of documents to skip | `0` | -| `max_hits` | `Integer` | Maximum number of hits to return (by default 20) | `20` | -| `search_field` | `[String]` | Fields to search on if no field name is specified in the query. Comma-separated list, e.g. "field1,field2" | index_config.search_settings.default_search_fields | -| `snippet_fields` | `[String]` | Fields to extract snippet on. Comma-separated list, e.g. "field1,field2" | | -| `sort_by` | `[String]` | Fields to sort the query results on. You can sort by one or two fast fields or by BM25 `_score` (requires fieldnorms). By default, hits are sorted by their document ID. | | -| `format` | `Enum` | The output format. Allowed values are "json" or "pretty_json" | `pretty_json` | -| `aggs` | `JSON` | The aggregations request. See the [aggregations doc](aggregation.md) for supported aggregations. | | +| Variable | Type | Description | Default value | +|---------------------|------------|-----------------|-----------------| +| `query` | `String` | Query text. See the [query language doc](query-language.md) | _required_ | +| `start_timestamp` | `i64` | If set, restrict search to documents with a `timestamp >= start_timestamp`, taking advantage of potential time pruning oportunities. The value must be in seconds. | | +| `end_timestamp` | `i64` | If set, restrict search to documents with a `timestamp < end_timestamp`, taking advantage of potential time pruning oportunities. The value must be in seconds. | | +| `start_offset` | `Integer` | Number of documents to skip | `0` | +| `max_hits` | `Integer` | Maximum number of hits to return (by default 20) | `20` | +| `search_field` | `[String]` | Fields to search on if no field name is specified in the query. Comma-separated list, e.g. "field1,field2" | index_config.search_settings.default_search_fields | +| `snippet_fields` | `[String]` | Fields to extract snippet on. Comma-separated list, e.g. "field1,field2" | | +| `sort_by` | `[String]` | Fields to sort the query results on. You can sort by one or two fast fields or by BM25 `_score` (requires fieldnorms). By default, hits are sorted by their document ID. | | +| `format` | `Enum` | The output format. Allowed values are "json" or "pretty_json" | `pretty_json` | +| `aggs` | `JSON` | The aggregations request. See the [aggregations doc](aggregation.md) for supported aggregations. | | :::info The `start_timestamp` and `end_timestamp` should be specified in seconds regardless of the timestamp field precision. @@ -144,14 +144,14 @@ The endpoint will return 10 million values if 10 million documents match the que #### Get parameters -| Variable | Type | Description | Default value | -|---------------------|------------|------------------------------------------------------------------------------------------------------------------|----------------------------------------------------| -| `query` | `String` | Query text. See the [query language doc](query-language.md) (mandatory) | | -| `fast_field` | `String` | Name of a field to retrieve from documents. This field must be a fast field of type `i64` or `u64`. (mandatory) | | -| `search_field` | `[String]` | Fields to search on. Comma-separated list, e.g. "field1,field2" | index_config.search_settings.default_search_fields | -| `start_timestamp` | `i64` | If set, restrict search to documents with a `timestamp >= start_timestamp`. The value must be in seconds. | | -| `end_timestamp` | `i64` | If set, restrict search to documents with a `timestamp < end_timestamp`. The value must be in seconds. | | -| `partition_by_field` | `String` | If set, the endpoint returns chunks of data for each partition field value. This field must be a fast field of type `i64` or `u64`. | | +| Variable | Type | Description | Default value | +|---------------------|------------|----------------------------------------------------------------------------------------------------------|----------------------------------------------------| +| `query` | `String` | Query text. See the [query language doc](query-language.md) | _required_ | +| `fast_field` | `String` | Name of a field to retrieve from documents. This field must be a fast field of type `i64` or `u64`. | _required_ | +| `search_field` | `[String]` | Fields to search on. Comma-separated list, e.g. "field1,field2" | index_config.search_settings.default_search_fields | +| `start_timestamp` | `i64` | If set, restrict search to documents with a `timestamp >= start_timestamp`. The value must be in seconds. | | +| `end_timestamp` | `i64` | If set, restrict search to documents with a `timestamp < end_timestamp`. The value must be in seconds. | | +| `partition_by_field` | `String` | If set, the endpoint returns chunks of data for each partition field value. This field must be a fast field of type `i64` or `u64`. | | | `output_format` | `String` | Response output format. `csv` or `clickHouseRowBinary` | `csv` | :::info @@ -223,16 +223,16 @@ The response is a JSON object, and the content type is `application/json; charse POST api/v1/indexes ``` -Create an index by posting an `IndexConfig` payload. The API accepts JSON with `content-type: application/json` and YAML `content-type: application/yaml`. +Create an index by posting an `IndexConfig` payload. The API accepts JSON with `content-type: application/json` and YAML with `content-type: application/yaml`. #### POST payload | Variable | Type | Description | Default value | |---------------------|--------------------|-----------------------------------------------------------------------------------------------------------------------|---------------------------------------| -| `version` | `String` | Config format version, use the same as your Quickwit version. (mandatory) | | -| `index_id` | `String` | Index ID, see its [validation rules](../configuration/index-config.md#index-id) on identifiers. (mandatory) | | +| `version` | `String` | Config format version, use the same as your Quickwit version. | _required_ | +| `index_id` | `String` | Index ID, see its [validation rules](../configuration/index-config.md#index-id) on identifiers. | _required_ | | `index_uri` | `String` | Defines where the index files are stored. This parameter expects a [storage URI](../configuration/storage-config.md#storage-uris). | `{default_index_root_uri}/{index_id}` | -| `doc_mapping` | `DocMapping` | Doc mapping object as specified in the [index config docs](../configuration/index-config.md#doc-mapping) (mandatory) | | +| `doc_mapping` | `DocMapping` | Doc mapping object as specified in the [index config docs](../configuration/index-config.md#doc-mapping). | _required_ | | `indexing_settings` | `IndexingSettings` | Indexing settings object as specified in the [index config docs](../configuration/index-config.md#indexing-settings). | | | `search_settings` | `SearchSettings` | Search settings object as specified in the [index config docs](../configuration/index-config.md#search-settings). | | | `retention` | `Retention` | Retention policy object as specified in the [index config docs](../configuration/index-config.md#retention-policy). | | @@ -240,11 +240,11 @@ Create an index by posting an `IndexConfig` payload. The API accepts JSON with ` **Payload Example** -curl -XPOST http://0.0.0.0:8080/api/v1/indexes --data @index_config.json -H "Content-Type: application/json" +curl -XPOST http://localhost:7280/api/v1/indexes --data @index_config.json -H "Content-Type: application/json" ```json title="index_config.json { - "version": "0.7", + "version": "0.8", "index_id": "hdfs-logs", "doc_mapping": { "field_mappings": [ @@ -285,9 +285,6 @@ curl -XPOST http://0.0.0.0:8080/api/v1/indexes --data @index_config.json -H "Con "max_merge_ops": 3, "merge_factor": 10, "max_merge_factor": 12 - }, - "resources": { - "max_merge_write_throughput": "80mb" } }, "retention": { @@ -301,62 +298,105 @@ curl -XPOST http://0.0.0.0:8080/api/v1/indexes --data @index_config.json -H "Con The response is the index metadata of the created index, and the content type is `application/json; charset=UTF-8.` -| Field | Description | Type | -|----------------------|-----------------------------------------|:---------------------:| -| `index_config` | The posted index config. | `IndexConfig` | -| `checkpoint` | Map of checkpoints by source. | `IndexCheckpoint` | -| `create_timestamp` | Index creation timestamp | `number` | -| `sources` | List of the index sources configurations. | `Array` | +| Field | Description | Type | +|----------------------|-----------------------------------------------|:---------------------:| +| `version` | The current index configuration format version. | `string` | +| `index_uid` | The server-generated index UID. | `string` | +| `index_config` | The posted index config. | `IndexConfig` | +| `checkpoint` | Map of checkpoints by source. | `IndexCheckpoint` | +| `create_timestamp` | Index creation timestamp | `number` | +| `sources` | List of the index sources configurations. | `Array` | -### Update an index (search settings and retention policy only) +### Update an index ``` PUT api/v1/indexes/ ``` -Updates the search settings and retention policy of an index. This endpoint follows PUT semantics (not PATCH), which means that all the updatable fields of the index configuration are replaced by the values specified in this request. In particular, omitting an optional field like retention_policy will delete the associated configuration. Unlike the create endpoint, this API only accepts JSON payloads. +Updates the configurations of an index. This endpoint follows PUT semantics, which means that all the fields of the current configuration are replaced by the values specified in this request or the associated defaults. In particular, if the field is optional (e.g. `retention_policy`), omitting it will delete the associated configuration. If the new configuration file contains updates that cannot be applied, the request fails, and none of the updates are applied. The API accepts JSON with `content-type: application/json` and YAML with `content-type: application/yaml`. + +- The retention policy update is automatically picked up by the janitor service on its next state refresh. +- The search settings update is automatically picked up by searcher nodes when the next query is executed. +- The indexing settings update is not automatically picked up by the indexer nodes, they need to be manually restarted. #### PUT payload | Variable | Type | Description | Default value | |---------------------|--------------------|-----------------------------------------------------------------------------------------------------------------------|---------------------------------------| +| `version` | `String` | Config format version, use the same as your Quickwit version. | _required_ | +| `index_id` | `String` | Index ID, must be the same index as in the request URI. | _required_ | +| `index_uri` | `String` | Defines where the index files are stored. Cannot be updated. | `{current_index_uri}` | +| `doc_mapping` | `DocMapping` | Doc mapping object as specified in the [index config docs](../configuration/index-config.md#doc-mapping). Cannot be updated. | _required_ | +| `indexing_settings` | `IndexingSettings` | Indexing settings object as specified in the [index config docs](../configuration/index-config.md#indexing-settings). | | | `search_settings` | `SearchSettings` | Search settings object as specified in the [index config docs](../configuration/index-config.md#search-settings). | | | `retention` | `Retention` | Retention policy object as specified in the [index config docs](../configuration/index-config.md#retention-policy). | | **Payload Example** -curl -XPUT http://0.0.0.0:8080/api/v1/indexes --data @index_update.json -H "Content-Type: application/json" +curl -XPUT http://localhost:7280/api/v1/indexes/hdfs-logs --data @updated_index_update.json -H "Content-Type: application/json" -```json title="index_update.json +```json title="updated_index_update.json { + "version": "0.8", + "index_id": "hdfs-logs", + "doc_mapping": { + "field_mappings": [ + { + "name": "tenant_id", + "type": "u64", + "fast": true + }, + { + "name": "app_id", + "type": "u64", + "fast": true + }, + { + "name": "timestamp", + "type": "datetime", + "input_formats": ["unix_timestamp"], + "fast_precision": "seconds", + "fast": true + }, + { + "name": "body", + "type": "text", + "record": "position" + } + ], + "partition_key": "tenant_id", + "max_num_partitions": 200, + "tag_fields": ["tenant_id"], + "timestamp_field": "timestamp" + }, "search_settings": { "default_search_fields": ["body"] }, + "indexing_settings": { + "merge_policy": { + "type": "limit_merge", + "max_merge_ops": 3, + "merge_factor": 10, + "max_merge_factor": 12 + } + }, "retention": { - "period": "3 days", + "period": "30 days", "schedule": "@daily" } } ``` -:::warning -Calling the update endpoint with the following payload will remove the current retention policy. -```json -{ - "search_settings": { - "default_search_fields": ["body"] - } -} -``` - #### Response The response is the index metadata of the updated index, and the content type is `application/json; charset=UTF-8.` | Field | Description | Type | |----------------------|-----------------------------------------|:---------------------:| +| `version` | The current server configuration version. | `string` | +| `index_uid` | The server-generated index UID. | `string` | | `index_config` | The posted index config. | `IndexConfig` | | `checkpoint` | Map of checkpoints by source. | `IndexCheckpoint` | | `create_timestamp` | Index creation timestamp | `number` | @@ -377,6 +417,8 @@ The response is the index metadata of the requested index, and the content type | Field | Description | Type | |----------------------|-------------------------------------------|:---------------------:| +| `version` | The current server configuration version. | `string` | +| `index_uid` | The server-generated index UID. | `string` | | `index_config` | The posted index config. | `IndexConfig` | | `checkpoint` | Map of checkpoints by source. | `IndexCheckpoint` | | `create_timestamp` | Index creation timestamp. | `number` | @@ -543,22 +585,22 @@ Create source by posting a source config JSON payload. #### POST payload -| Variable | Type | Description | Default value | -|-------------------|----------|------------------------------------------------------------------------------------------------------|---------------| -| `version** | `String` | Config format version, put your current Quickwit version. (mandatory) | | -| `source_id` | `String` | Source ID. See ID [validation rules](../configuration/source-config.md)(mandatory) | | -| `source_type` | `String` | Source type: `kafka`, `kinesis` or `pulsar` (mandatory) | | -| `num_pipelines` | `usize` | Number of running indexing pipelines per node for this source. | 1 | -| `params` | `object` | Source parameters as defined in [source config docs](../configuration/source-config.md). (mandatory) | | +| Variable | Type | Description | Default value | +|-------------------|----------|----------------------------------------------------------------------------------------|---------------| +| `version** | `String` | Config format version, put your current Quickwit version. | _required_ | +| `source_id` | `String` | Source ID. See ID [validation rules](../configuration/source-config.md). | _required_ | +| `source_type` | `String` | Source type: `kafka`, `kinesis` or `pulsar`. | _required_ | +| `num_pipelines` | `usize` | Number of running indexing pipelines per node for this source. | 1 | +| `params` | `object` | Source parameters as defined in [source config docs](../configuration/source-config.md). | _required_ | **Payload Example** -curl -XPOST http://0.0.0.0:8080/api/v1/indexes/my-index/sources --data @source_config.json -H "Content-Type: application/json" +curl -XPOST http://localhost:7280/api/v1/indexes/my-index/sources --data @source_config.json -H "Content-Type: application/json" ```json title="source_config.json { - "version": "0.7", + "version": "0.8", "source_id": "kafka-source", "source_type": "kafka", "params": { @@ -647,9 +689,9 @@ The endpoint simply appends your delete task to the delete task queue in the met #### POST payload `DeleteQuery` -| Variable | Type | Description | Default value | -|---------------------|------------|-----------------------------------------------------------------------------------------------------------|----------------------------------------------------| -| `query` | `String` | Query text. See the [query language doc](query-language.md) (mandatory) | | +| Variable | Type | Description | Default value | +|---------------------|------------|---------------------------------------------------------------------------------------------------------|----------------------------------------------------| +| `query` | `String` | Query text. See the [query language doc](query-language.md) | _required_ | | `search_field` | `[String]` | Fields to search on. Comma-separated list, e.g. "field1,field2" | index_config.search_settings.default_search_fields | | `start_timestamp` | `i64` | If set, restrict search to documents with a `timestamp >= start_timestamp`. The value must be in seconds. | | | `end_timestamp` | `i64` | If set, restrict search to documents with a `timestamp < end_timestamp`. The value must be in seconds. | | diff --git a/quickwit/Cargo.lock b/quickwit/Cargo.lock index 1434a5913d8..6ec1c8a07f5 100644 --- a/quickwit/Cargo.lock +++ b/quickwit/Cargo.lock @@ -654,9 +654,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.5.0" +version = "1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9ac79e9f3a4d576f3cd4a470a0275b138d9e7b11b1cd514a6858ae0a79dd5bb" +checksum = "8508de54f34b8feca6638466c2bd2de9d1df5bf79c578de9a649b72d644006b3" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -669,6 +669,7 @@ dependencies = [ "http 0.2.12", "http-body 0.4.6", "http-body 1.0.0", + "httparse", "hyper 0.14.28", "hyper-rustls", "indexmap 2.1.0", @@ -5765,7 +5766,6 @@ dependencies = [ "anyhow", "async-trait", "bytesize", - "dyn-clone", "futures", "http 0.2.12", "mockall", @@ -6035,7 +6035,6 @@ dependencies = [ "async-trait", "bytes", "bytesize", - "dyn-clone", "fail", "futures", "http 0.2.12", @@ -6280,7 +6279,6 @@ dependencies = [ "bytes", "bytesize", "bytestring", - "dyn-clone", "futures", "glob", "http 0.2.12", diff --git a/quickwit/quickwit-cli/src/index/mod.rs b/quickwit/quickwit-cli/src/index.rs similarity index 94% rename from quickwit/quickwit-cli/src/index/mod.rs rename to quickwit/quickwit-cli/src/index.rs index efc253b9667..9028fa26c56 100644 --- a/quickwit/quickwit-cli/src/index/mod.rs +++ b/quickwit/quickwit-cli/src/index.rs @@ -57,13 +57,10 @@ use tabled::{Table, Tabled}; use thousands::Separable; use tracing::{debug, Level}; -use self::update::{build_index_update_command, IndexUpdateCliCommand}; use crate::checklist::GREEN_COLOR; use crate::stats::{mean, percentile, std_deviation}; use crate::{client_args, make_table, prompt_confirmation, ClientArgs, THROUGHPUT_WINDOW_SIZE}; -pub mod update; - pub fn build_index_command() -> Command { Command::new("index") .about("Manages indexes: creates, updates, deletes, ingests, searches, describes...") @@ -81,7 +78,18 @@ pub fn build_index_command() -> Command { ]) ) .subcommand( - build_index_update_command().display_order(2) + Command::new("update") + .display_order(1) + .about("Updates an index using an index config file.") + .long_about("This command follows PUT semantics, which means that all the fields of the current configuration are replaced by the values specified in this request or the associated defaults. In particular, if the field is optional (e.g. `retention_policy`), omitting it will delete the associated configuration. If the new configuration file contains updates that cannot be applied, the request fails, and none of the updates are applied.") + .args(&[ + arg!(--index "ID of the target index") + .display_order(1) + .required(true), + arg!(--"index-config" "Location of the index config file.") + .display_order(2) + .required(true), + ]) ) .subcommand( Command::new("clear") @@ -213,6 +221,14 @@ pub struct CreateIndexArgs { pub assume_yes: bool, } +#[derive(Debug, Eq, PartialEq)] +pub struct UpdateIndexArgs { + pub client_args: ClientArgs, + pub index_id: IndexId, + pub index_config_uri: Uri, + pub assume_yes: bool, +} + #[derive(Debug, Eq, PartialEq)] pub struct DescribeIndexArgs { pub client_args: ClientArgs, @@ -260,12 +276,12 @@ pub struct ListIndexesArgs { pub enum IndexCliCommand { Clear(ClearIndexArgs), Create(CreateIndexArgs), + Update(UpdateIndexArgs), Delete(DeleteIndexArgs), Describe(DescribeIndexArgs), Ingest(IngestDocsArgs), List(ListIndexesArgs), Search(SearchIndexArgs), - Update(IndexUpdateCliCommand), } impl IndexCliCommand { @@ -288,7 +304,7 @@ impl IndexCliCommand { "ingest" => Self::parse_ingest_args(submatches), "list" => Self::parse_list_args(submatches), "search" => Self::parse_search_args(submatches), - "update" => Ok(Self::Update(IndexUpdateCliCommand::parse_args(submatches)?)), + "update" => Self::parse_update_args(submatches), _ => bail!("unknown index subcommand `{subcommand}`"), } } @@ -323,6 +339,25 @@ impl IndexCliCommand { })) } + fn parse_update_args(mut matches: ArgMatches) -> anyhow::Result { + let client_args = ClientArgs::parse(&mut matches)?; + let index_id = matches + .remove_one::("index") + .expect("`index` should be a required arg."); + let index_config_uri = matches + .remove_one::("index-config") + .map(|uri| Uri::from_str(&uri)) + .expect("`index-config` should be a required arg.")?; + let assume_yes = matches.get_flag("yes"); + + Ok(Self::Update(UpdateIndexArgs { + index_id, + client_args, + index_config_uri, + assume_yes, + })) + } + fn parse_describe_args(mut matches: ArgMatches) -> anyhow::Result { let client_args = ClientArgs::parse(&mut matches)?; let index_id = matches @@ -449,7 +484,7 @@ impl IndexCliCommand { Self::Ingest(args) => ingest_docs_cli(args).await, Self::List(args) => list_index_cli(args).await, Self::Search(args) => search_index_cli(args).await, - Self::Update(args) => args.execute().await, + Self::Update(args) => update_index_cli(args).await, } } } @@ -501,6 +536,35 @@ pub async fn create_index_cli(args: CreateIndexArgs) -> anyhow::Result<()> { Ok(()) } +pub async fn update_index_cli(args: UpdateIndexArgs) -> anyhow::Result<()> { + debug!(args=?args, "update-index"); + println!("❯ Updating index..."); + let storage_resolver = StorageResolver::unconfigured(); + let file_content = load_file(&storage_resolver, &args.index_config_uri).await?; + let index_config_str = std::str::from_utf8(&file_content) + .with_context(|| { + format!( + "index config file `{}` contains some invalid UTF-8 characters", + args.index_config_uri + ) + })? + .to_string(); + let config_format = ConfigFormat::sniff_from_uri(&args.index_config_uri)?; + let qw_client = args.client_args.client(); + if !args.assume_yes { + let prompt = "This operation will update the index configuration. Do you want to proceed?"; + if !prompt_confirmation(prompt, false) { + return Ok(()); + } + } + qw_client + .indexes() + .update(&args.index_id, &index_config_str, config_format) + .await?; + println!("{} Index successfully updated.", "✔".color(GREEN_COLOR)); + Ok(()) +} + pub async fn list_index_cli(args: ListIndexesArgs) -> anyhow::Result<()> { debug!(args=?args, "list-index"); let qw_client = args.client_args.client(); diff --git a/quickwit/quickwit-cli/src/index/update.rs b/quickwit/quickwit-cli/src/index/update.rs deleted file mode 100644 index f417dbb0216..00000000000 --- a/quickwit/quickwit-cli/src/index/update.rs +++ /dev/null @@ -1,257 +0,0 @@ -// Copyright (C) 2024 Quickwit, Inc. -// -// Quickwit is offered under the AGPL v3.0 and as commercial software. -// For commercial licensing, contact us at hello@quickwit.io. -// -// AGPL: -// This program is free software: you can redistribute it and/or modify -// it under the terms of the GNU Affero General Public License as -// published by the Free Software Foundation, either version 3 of the -// License, or (at your option) any later version. -// -// This program is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Affero General Public License for more details. -// -// You should have received a copy of the GNU Affero General Public License -// along with this program. If not, see . - -use anyhow::{bail, Context}; -use clap::{arg, ArgMatches, Command}; -use colored::Colorize; -use quickwit_config::{RetentionPolicy, SearchSettings}; -use quickwit_proto::types::IndexId; -use quickwit_serve::IndexUpdates; -use tracing::debug; - -use crate::checklist::GREEN_COLOR; -use crate::ClientArgs; - -pub fn build_index_update_command() -> Command { - Command::new("update") - .subcommand_required(true) - .subcommand( - Command::new("search-settings") - .about("Updates default search settings.") - .args(&[ - arg!(--index "ID of the target index") - .display_order(1) - .required(true), - arg!(--"default-search-fields" "List of fields that Quickwit will search into if the user query does not explicitly target a field. Space-separated list, e.g. \"field1 field2\". If no value is provided, existing defaults are removed and queries without target field will fail.") - .display_order(2) - .num_args(0..) - .required(true), - ])) - .subcommand( - Command::new("retention-policy") - .about("Configures or disables the retention policy.") - .args(&[ - arg!(--index "ID of the target index") - .display_order(1) - .required(true), - arg!(--"period" "Duration after which splits are dropped. Expressed in a human-readable way (`1 day`, `2 hours`, `1 week`, ...)") - .display_order(2) - .required(false), - arg!(--"schedule" "Frequency at which the retention policy is evaluated and applied. Expressed as a cron expression (0 0 * * * *) or human-readable form (hourly, daily, weekly, ...).") - .display_order(3) - .required(false), - arg!(--"disable" "Disables the retention policy. Old indexed data will not be cleaned up anymore.") - .display_order(4) - .required(false), - ]) - ) -} - -#[derive(Debug, Eq, PartialEq)] -pub struct RetentionPolicyArgs { - pub client_args: ClientArgs, - pub index_id: IndexId, - pub disable: bool, - pub period: Option, - pub schedule: Option, -} - -#[derive(Debug, Eq, PartialEq)] -pub struct SearchSettingsArgs { - pub client_args: ClientArgs, - pub index_id: IndexId, - pub default_search_fields: Vec, -} - -#[derive(Debug, Eq, PartialEq)] -pub enum IndexUpdateCliCommand { - RetentionPolicy(RetentionPolicyArgs), - SearchSettings(SearchSettingsArgs), -} - -impl IndexUpdateCliCommand { - pub fn parse_args(mut matches: ArgMatches) -> anyhow::Result { - let (subcommand, submatches) = matches - .remove_subcommand() - .context("failed to parse index update subcommand")?; - match subcommand.as_str() { - "retention-policy" => Self::parse_update_retention_policy_args(submatches), - "search-settings" => Self::parse_update_search_settings_args(submatches), - _ => bail!("unknown index update subcommand `{subcommand}`"), - } - } - - fn parse_update_retention_policy_args(mut matches: ArgMatches) -> anyhow::Result { - let client_args = ClientArgs::parse(&mut matches)?; - let index_id = matches - .remove_one::("index") - .expect("`index` should be a required arg."); - let disable = matches.get_flag("disable"); - let period = matches.remove_one::("period"); - let schedule = matches.remove_one::("schedule"); - Ok(Self::RetentionPolicy(RetentionPolicyArgs { - client_args, - index_id, - disable, - period, - schedule, - })) - } - - fn parse_update_search_settings_args(mut matches: ArgMatches) -> anyhow::Result { - let client_args = ClientArgs::parse(&mut matches)?; - let index_id = matches - .remove_one::("index") - .expect("`index` should be a required arg."); - let default_search_fields = matches - .remove_many::("default-search-fields") - .map(|values| values.collect()) - // --default-search-fields should be made optional if other fields - // are added to SearchSettings - .expect("`default-search-fields` should be a required arg."); - Ok(Self::SearchSettings(SearchSettingsArgs { - client_args, - index_id, - default_search_fields, - })) - } - - pub async fn execute(self) -> anyhow::Result<()> { - match self { - Self::RetentionPolicy(args) => update_retention_policy_cli(args).await, - Self::SearchSettings(args) => update_search_settings_cli(args).await, - } - } -} - -pub async fn update_retention_policy_cli(args: RetentionPolicyArgs) -> anyhow::Result<()> { - debug!(args=?args, "update-index-retention-policy"); - println!("❯ Updating index retention policy..."); - let qw_client = args.client_args.client(); - let metadata = qw_client.indexes().get(&args.index_id).await?; - let new_retention_policy_opt = match ( - args.disable, - args.period, - args.schedule, - metadata.index_config.retention_policy_opt, - ) { - (true, Some(_), Some(_), _) | (true, None, Some(_), _) | (true, Some(_), None, _) => { - bail!("`--period` and `--schedule` cannot be used together with `--disable`") - } - (false, None, None, _) => bail!("either `--period` or `--disable` must be specified"), - (false, None, Some(_), None) => { - bail!("`--period` is required when creating a retention policy") - } - (true, None, None, _) => None, - (false, None, Some(schedule), Some(policy)) => Some(RetentionPolicy { - retention_period: policy.retention_period, - evaluation_schedule: schedule, - }), - (false, Some(period), schedule_opt, None) => Some(RetentionPolicy { - retention_period: period, - evaluation_schedule: schedule_opt.unwrap_or(RetentionPolicy::default_schedule()), - }), - (false, Some(period), schedule_opt, Some(policy)) => Some(RetentionPolicy { - retention_period: period, - evaluation_schedule: schedule_opt.unwrap_or(policy.evaluation_schedule.clone()), - }), - }; - if let Some(new_retention_policy) = new_retention_policy_opt.as_ref() { - println!( - "New retention policy: {}", - serde_json::to_string(&new_retention_policy)? - ); - } else { - println!("Disable retention policy."); - } - qw_client - .indexes() - .update( - &args.index_id, - IndexUpdates { - retention_policy_opt: new_retention_policy_opt, - search_settings: metadata.index_config.search_settings, - }, - ) - .await?; - println!("{} Index successfully updated.", "✔".color(GREEN_COLOR)); - Ok(()) -} - -pub async fn update_search_settings_cli(args: SearchSettingsArgs) -> anyhow::Result<()> { - debug!(args=?args, "update-index-search-settings"); - println!("❯ Updating index search settings..."); - let qw_client = args.client_args.client(); - let metadata = qw_client.indexes().get(&args.index_id).await?; - let search_settings = SearchSettings { - default_search_fields: args.default_search_fields, - }; - println!( - "New search settings: {}", - serde_json::to_string(&search_settings)? - ); - qw_client - .indexes() - .update( - &args.index_id, - IndexUpdates { - retention_policy_opt: metadata.index_config.retention_policy_opt, - search_settings, - }, - ) - .await?; - println!("{} Index successfully updated.", "✔".color(GREEN_COLOR)); - Ok(()) -} - -#[cfg(test)] -mod test { - use super::*; - use crate::cli::{build_cli, CliCommand}; - use crate::index::IndexCliCommand; - - #[test] - fn test_cmd_update_subsubcommand() { - let app = build_cli().no_binary_name(true); - let matches = app - .try_get_matches_from([ - "index", - "update", - "retention-policy", - "--index", - "my-index", - "--period", - "1 day", - ]) - .unwrap(); - let command = CliCommand::parse_cli_args(matches).unwrap(); - assert!(matches!( - command, - CliCommand::Index(IndexCliCommand::Update( - IndexUpdateCliCommand::RetentionPolicy(RetentionPolicyArgs { - client_args: _, - index_id, - disable: false, - period: Some(period), - schedule: None, - }) - )) if &index_id == "my-index" && &period == "1 day" - )); - } -} diff --git a/quickwit/quickwit-cli/src/tool.rs b/quickwit/quickwit-cli/src/tool.rs index 2d894a7eb09..7b1e6ad9435 100644 --- a/quickwit/quickwit-cli/src/tool.rs +++ b/quickwit/quickwit-cli/src/tool.rs @@ -722,7 +722,7 @@ async fn extract_split_cli(args: ExtractSplitArgs) -> anyhow::Result<()> { let config = load_node_config(&args.config_uri).await?; let (storage_resolver, metastore_resolver) = get_resolvers(&config.storage_configs, &config.metastore_configs); - let mut metastore = metastore_resolver.resolve(&config.metastore_uri).await?; + let metastore = metastore_resolver.resolve(&config.metastore_uri).await?; let index_metadata = metastore .index_metadata(IndexMetadataRequest::for_index_id(args.index_id)) .await? diff --git a/quickwit/quickwit-cli/tests/cli.rs b/quickwit/quickwit-cli/tests/cli.rs index 848934c6a28..628c695be78 100644 --- a/quickwit/quickwit-cli/tests/cli.rs +++ b/quickwit/quickwit-cli/tests/cli.rs @@ -22,25 +22,21 @@ mod helpers; use std::path::Path; -use std::str::FromStr; use anyhow::Result; use clap::error::ErrorKind; use helpers::{TestEnv, TestStorageType}; use quickwit_cli::checklist::ChecklistError; use quickwit_cli::cli::build_cli; -use quickwit_cli::index::update::{update_retention_policy_cli, RetentionPolicyArgs}; use quickwit_cli::index::{ - create_index_cli, delete_index_cli, search_index, CreateIndexArgs, DeleteIndexArgs, - SearchIndexArgs, + create_index_cli, delete_index_cli, search_index, update_index_cli, CreateIndexArgs, + DeleteIndexArgs, SearchIndexArgs, UpdateIndexArgs, }; use quickwit_cli::tool::{ garbage_collect_index_cli, local_ingest_docs_cli, GarbageCollectIndexArgs, LocalIngestDocsArgs, }; -use quickwit_cli::ClientArgs; use quickwit_common::fs::get_cache_directory_path; use quickwit_common::rand::append_random_suffix; -use quickwit_common::uri::Uri; use quickwit_config::{RetentionPolicy, SourceInputFormat, CLI_SOURCE_ID}; use quickwit_metastore::{ ListSplitsRequestExt, MetastoreResolver, MetastoreServiceExt, MetastoreServiceStreamSplitsExt, @@ -57,11 +53,8 @@ use crate::helpers::{create_test_env, upload_test_file, PACKAGE_BIN_NAME}; async fn create_logs_index(test_env: &TestEnv) -> anyhow::Result<()> { let args = CreateIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, - index_config_uri: test_env.index_config_uri.clone(), + client_args: test_env.default_client_args(), + index_config_uri: test_env.resource_files.index_config.clone(), overwrite: false, assume_yes: true, }; @@ -70,7 +63,7 @@ async fn create_logs_index(test_env: &TestEnv) -> anyhow::Result<()> { async fn local_ingest_docs(input_path: &Path, test_env: &TestEnv) -> anyhow::Result<()> { let args = LocalIngestDocsArgs { - config_uri: test_env.config_uri.clone(), + config_uri: test_env.resource_files.config.clone(), index_id: test_env.index_id.clone(), input_path_opt: Some(input_path.to_path_buf()), input_format: SourceInputFormat::Json, @@ -118,12 +111,9 @@ async fn test_cmd_create_no_index_uri() { .unwrap(); test_env.start_server().await.unwrap(); - let index_config_without_uri = Uri::from_str(&test_env.index_config_without_uri()).unwrap(); + let index_config_without_uri = test_env.resource_files.index_config_without_uri.clone(); let args = CreateIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_config_uri: index_config_without_uri, overwrite: false, assume_yes: true, @@ -146,12 +136,9 @@ async fn test_cmd_create_overwrite() { .unwrap(); test_env.start_server().await.unwrap(); - let index_config_without_uri = Uri::from_str(&test_env.index_config_without_uri()).unwrap(); + let index_config_without_uri = test_env.resource_files.index_config_without_uri.clone(); let args = CreateIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_config_uri: index_config_without_uri, overwrite: true, assume_yes: true, @@ -182,9 +169,9 @@ async fn test_cmd_ingest_on_non_existing_index() { .unwrap(); let args = LocalIngestDocsArgs { - config_uri: test_env.config_uri, + config_uri: test_env.resource_files.config, index_id: "index-does-not-exist".to_string(), - input_path_opt: Some(test_env.resource_files["logs"].clone()), + input_path_opt: Some(test_env.resource_files.log_docs.clone()), input_format: SourceInputFormat::Json, overwrite: false, clear_cache: true, @@ -212,9 +199,9 @@ async fn test_ingest_docs_cli_keep_cache() { create_logs_index(&test_env).await.unwrap(); let args = LocalIngestDocsArgs { - config_uri: test_env.config_uri, + config_uri: test_env.resource_files.config, index_id, - input_path_opt: Some(test_env.resource_files["logs"].clone()), + input_path_opt: Some(test_env.resource_files.log_docs.clone()), input_format: SourceInputFormat::Json, overwrite: false, clear_cache: false, @@ -239,9 +226,9 @@ async fn test_ingest_docs_cli() { let index_uid = test_env.index_metadata().await.unwrap().index_uid; let args = LocalIngestDocsArgs { - config_uri: test_env.config_uri.clone(), + config_uri: test_env.resource_files.config.clone(), index_id: index_id.clone(), - input_path_opt: Some(test_env.resource_files["logs"].clone()), + input_path_opt: Some(test_env.resource_files.log_docs.clone()), input_format: SourceInputFormat::Json, overwrite: false, clear_cache: true, @@ -270,7 +257,7 @@ async fn test_ingest_docs_cli() { // Ingest a non-existing file should fail. let args = LocalIngestDocsArgs { - config_uri: test_env.config_uri, + config_uri: test_env.resource_files.config, index_id: test_env.index_id, input_path_opt: Some(test_env.data_dir_path.join("file-does-not-exist.json")), input_format: SourceInputFormat::Json, @@ -345,7 +332,7 @@ async fn test_cmd_search_aggregation() { test_env.start_server().await.unwrap(); create_logs_index(&test_env).await.unwrap(); - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); @@ -371,7 +358,7 @@ async fn test_cmd_search_aggregation() { // search with aggregation let args = SearchIndexArgs { - index_id: test_env.index_id, + index_id: test_env.index_id.clone(), query: "paris OR tokio OR london".to_string(), aggregation: Some(serde_json::to_string(&aggregation).unwrap()), max_hits: 10, @@ -380,10 +367,7 @@ async fn test_cmd_search_aggregation() { snippet_fields: None, start_timestamp: None, end_timestamp: None, - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint, - ..Default::default() - }, + client_args: test_env.default_client_args(), sort_by_score: false, }; let search_response = search_index(args).await.unwrap(); @@ -448,13 +432,13 @@ async fn test_cmd_search_with_snippets() -> Result<()> { test_env.start_server().await.unwrap(); create_logs_index(&test_env).await.unwrap(); - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); // search with snippets let args = SearchIndexArgs { - index_id: test_env.index_id, + index_id: test_env.index_id.clone(), query: "event:baz".to_string(), aggregation: None, max_hits: 10, @@ -463,10 +447,7 @@ async fn test_cmd_search_with_snippets() -> Result<()> { snippet_fields: Some(vec!["event".to_string()]), start_timestamp: None, end_timestamp: None, - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint, - ..Default::default() - }, + client_args: test_env.default_client_args(), sort_by_score: false, }; let search_response = search_index(args).await.unwrap(); @@ -493,10 +474,7 @@ async fn test_search_index_cli() { create_logs_index(&test_env).await.unwrap(); let create_search_args = |query: &str| SearchIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_id: index_id.clone(), query: query.to_string(), aggregation: None, @@ -509,7 +487,7 @@ async fn test_search_index_cli() { sort_by_score: false, }; - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); @@ -544,20 +522,16 @@ async fn test_cmd_update_index() { test_env.start_server().await.unwrap(); create_logs_index(&test_env).await.unwrap(); - // add a policy - update_retention_policy_cli(RetentionPolicyArgs { + // add retention policy + let args = UpdateIndexArgs { + client_args: test_env.default_client_args(), index_id: index_id.clone(), - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, - disable: false, - period: Some(String::from("1 week")), - schedule: Some(String::from("daily")), - }) - .await - .unwrap(); + index_config_uri: test_env.resource_files.index_config_with_retention.clone(), + assume_yes: true, + }; + update_index_cli(args).await.unwrap(); let index_metadata = test_env.index_metadata().await.unwrap(); + assert_eq!(index_metadata.index_id(), test_env.index_id); assert_eq!( index_metadata.index_config.retention_policy_opt, Some(RetentionPolicy { @@ -566,34 +540,16 @@ async fn test_cmd_update_index() { }) ); - // invalid args - update_retention_policy_cli(RetentionPolicyArgs { - index_id: index_id.clone(), - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, - disable: true, - period: Some(String::from("a week")), - schedule: Some(String::from("daily")), - }) - .await - .unwrap_err(); - - // remove the policy - update_retention_policy_cli(RetentionPolicyArgs { + // remove retention policy + let args = UpdateIndexArgs { + client_args: test_env.default_client_args(), index_id, - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, - disable: true, - period: None, - schedule: None, - }) - .await - .unwrap(); + index_config_uri: test_env.resource_files.index_config.clone(), + assume_yes: true, + }; + update_index_cli(args).await.unwrap(); let index_metadata = test_env.index_metadata().await.unwrap(); + assert_eq!(index_metadata.index_id(), test_env.index_id); assert_eq!(index_metadata.index_config.retention_policy_opt, None); } @@ -620,10 +576,7 @@ async fn test_delete_index_cli_dry_run() { }; let create_delete_args = |dry_run| DeleteIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_id: index_id.clone(), dry_run, assume_yes: true, @@ -647,7 +600,7 @@ async fn test_delete_index_cli_dry_run() { .unwrap(); assert!(metastore.index_exists(&index_id).await.unwrap()); - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); @@ -673,15 +626,12 @@ async fn test_delete_index_cli() { test_env.start_server().await.unwrap(); create_logs_index(&test_env).await.unwrap(); - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); let args = DeleteIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_id: index_id.clone(), assume_yes: true, dry_run: false, @@ -702,11 +652,11 @@ async fn test_garbage_collect_cli_no_grace() { test_env.start_server().await.unwrap(); create_logs_index(&test_env).await.unwrap(); let index_uid = test_env.index_metadata().await.unwrap().index_uid; - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); - let mut metastore = MetastoreResolver::unconfigured() + let metastore = MetastoreResolver::unconfigured() .resolve(&test_env.metastore_uri) .await .unwrap(); @@ -724,7 +674,7 @@ async fn test_garbage_collect_cli_no_grace() { }; let create_gc_args = |dry_run| GarbageCollectIndexArgs { - config_uri: test_env.config_uri.clone(), + config_uri: test_env.resource_files.config.clone(), index_id: index_id.clone(), grace_period: Duration::from_secs(3600), dry_run, @@ -748,7 +698,7 @@ async fn test_garbage_collect_cli_no_grace() { assert_eq!(index_path.try_exists().unwrap(), true); let split_ids = vec![splits_metadata[0].split_id().to_string()]; - let mut metastore = refresh_metastore(metastore).await.unwrap(); + let metastore = refresh_metastore(metastore).await.unwrap(); let mark_for_deletion_request = MarkSplitsForDeletionRequest::new(index_uid.clone(), split_ids.clone()); metastore @@ -778,7 +728,7 @@ async fn test_garbage_collect_cli_no_grace() { assert_eq!(split_filepath.try_exists().unwrap(), false); } - let mut metastore = refresh_metastore(metastore).await.unwrap(); + let metastore = refresh_metastore(metastore).await.unwrap(); assert_eq!( metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) @@ -792,10 +742,7 @@ async fn test_garbage_collect_cli_no_grace() { ); let args = DeleteIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_id, dry_run: false, assume_yes: true, @@ -815,7 +762,7 @@ async fn test_garbage_collect_index_cli() { test_env.start_server().await.unwrap(); create_logs_index(&test_env).await.unwrap(); let index_uid = test_env.index_metadata().await.unwrap().index_uid; - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); @@ -832,13 +779,13 @@ async fn test_garbage_collect_index_cli() { }; let create_gc_args = |grace_period_secs| GarbageCollectIndexArgs { - config_uri: test_env.config_uri.clone(), + config_uri: test_env.resource_files.config.clone(), index_id: index_id.clone(), grace_period: Duration::from_secs(grace_period_secs), dry_run: false, }; - let mut metastore = MetastoreResolver::unconfigured() + let metastore = MetastoreResolver::unconfigured() .resolve(&test_env.metastore_uri) .await .unwrap(); @@ -862,7 +809,7 @@ async fn test_garbage_collect_index_cli() { garbage_collect_index_cli(args).await.unwrap(); // Split should still exists within grace period. - let mut metastore = refresh_metastore(metastore).await.unwrap(); + let metastore = refresh_metastore(metastore).await.unwrap(); let splits_metadata = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await @@ -901,7 +848,7 @@ async fn test_garbage_collect_index_cli() { .unwrap(); assert_eq!(split_path.try_exists().unwrap(), true); - let mut metastore = refresh_metastore(metastore).await.unwrap(); + let metastore = refresh_metastore(metastore).await.unwrap(); let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await @@ -917,7 +864,7 @@ async fn test_garbage_collect_index_cli() { assert_eq!(split_path.try_exists().unwrap(), true); // Staged splits should still exist within grace period. - let mut metastore = refresh_metastore(metastore).await.unwrap(); + let metastore = refresh_metastore(metastore).await.unwrap(); let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap()) .await @@ -936,7 +883,7 @@ async fn test_garbage_collect_index_cli() { garbage_collect_index_cli(args).await.unwrap(); - let mut metastore = refresh_metastore(metastore).await.unwrap(); + let metastore = refresh_metastore(metastore).await.unwrap(); let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) .await @@ -967,7 +914,7 @@ async fn test_all_local_index() { .unwrap(); assert!(metadata_file_exists); - local_ingest_docs(test_env.resource_files["logs"].as_path(), &test_env) + local_ingest_docs(test_env.resource_files.log_docs.as_path(), &test_env) .await .unwrap(); @@ -997,10 +944,7 @@ async fn test_all_local_index() { assert_eq!(search_stream_response, "72057597000000\n72057608000000\n"); let args = DeleteIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_id, dry_run: false, assume_yes: true, @@ -1028,7 +972,7 @@ async fn test_all_with_s3_localstack_cli() { let s3_path = upload_test_file( test_env.storage_resolver.clone(), - test_env.resource_files["logs"].clone(), + test_env.resource_files.log_docs.clone(), "quickwit-integration-tests", "sources/", &append_random_suffix("test-all--cli-s3-localstack"), @@ -1039,10 +983,7 @@ async fn test_all_with_s3_localstack_cli() { // Cli search let args = SearchIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_id: index_id.clone(), query: "level:info".to_string(), aggregation: None, @@ -1072,10 +1013,7 @@ async fn test_all_with_s3_localstack_cli() { assert_eq!(result["num_hits"], Value::Number(Number::from(2i64))); let args = DeleteIndexArgs { - client_args: ClientArgs { - cluster_endpoint: test_env.cluster_endpoint.clone(), - ..Default::default() - }, + client_args: test_env.default_client_args(), index_id: index_id.clone(), dry_run: false, assume_yes: true, diff --git a/quickwit/quickwit-cli/tests/helpers.rs b/quickwit/quickwit-cli/tests/helpers.rs index 1ff7fb8234e..0a52f6b9792 100644 --- a/quickwit/quickwit-cli/tests/helpers.rs +++ b/quickwit/quickwit-cli/tests/helpers.rs @@ -18,7 +18,6 @@ // along with this program. If not, see . use std::borrow::Borrow; -use std::collections::HashMap; use std::fs; use std::path::PathBuf; use std::str::FromStr; @@ -27,6 +26,7 @@ use std::sync::Arc; use anyhow::Context; use predicates::str; use quickwit_cli::service::RunCliCommand; +use quickwit_cli::ClientArgs; use quickwit_common::net::find_available_tcp_port; use quickwit_common::test_utils::wait_for_server_ready; use quickwit_common::uri::Uri; @@ -81,6 +81,12 @@ const DEFAULT_INDEX_CONFIG: &str = r#" default_search_fields: [event] "#; +const RETENTION_CONFIG: &str = r#" + retention: + period: 1 week + schedule: daily +"#; + const DEFAULT_QUICKWIT_CONFIG: &str = r#" version: 0.8 metastore_uri: #metastore_uri @@ -103,6 +109,15 @@ const WIKI_JSON_DOCS: &str = r#"{"body": "foo", "title": "shimroy", "url": "http {"body": "biz", "title": "modern", "url": "https://wiki.com?id=13"} "#; +pub struct TestResourceFiles { + pub config: Uri, + pub index_config: Uri, + pub index_config_without_uri: Uri, + pub index_config_with_retention: Uri, + pub log_docs: PathBuf, + pub wikipedia_docs: PathBuf, +} + /// A struct to hold few info about the test environment. pub struct TestEnv { /// The temporary directory of the test. @@ -112,14 +127,14 @@ pub struct TestEnv { /// Path of the directory where indexes are stored. pub indexes_dir_path: PathBuf, /// Resource files needed for the test. - pub resource_files: HashMap<&'static str, PathBuf>, + pub resource_files: TestResourceFiles, /// The metastore URI. pub metastore_uri: Uri, pub metastore_resolver: MetastoreResolver, pub metastore: MetastoreServiceClient, - pub config_uri: Uri, + pub cluster_endpoint: Url, - pub index_config_uri: Uri, + /// The index ID. pub index_id: IndexId, pub index_uri: Uri, @@ -137,12 +152,6 @@ impl TestEnv { .unwrap() } - pub fn index_config_without_uri(&self) -> String { - self.resource_files["index_config_without_uri"] - .display() - .to_string() - } - pub async fn index_metadata(&self) -> anyhow::Result { let index_metadata = self .metastore() @@ -155,7 +164,7 @@ impl TestEnv { pub async fn start_server(&self) -> anyhow::Result<()> { let run_command = RunCliCommand { - config_uri: self.config_uri.clone(), + config_uri: self.resource_files.config.clone(), services: Some(QuickwitService::supported_services()), }; tokio::spawn(async move { @@ -169,6 +178,13 @@ impl TestEnv { wait_for_server_ready(([127, 0, 0, 1], self.rest_listen_port).into()).await?; Ok(()) } + + pub fn default_client_args(&self) -> ClientArgs { + ClientArgs { + cluster_endpoint: self.cluster_endpoint.clone(), + ..Default::default() + } + } } pub enum TestStorageType { @@ -176,6 +192,10 @@ pub enum TestStorageType { LocalFileSystem, } +fn uri_from_path(path: PathBuf) -> Uri { + Uri::from_str(&format!("file://{}", path.display())).unwrap() +} + /// Creates all necessary artifacts in a test environment. pub async fn create_test_env( index_id: IndexId, @@ -216,6 +236,14 @@ pub async fn create_test_env( .replace("#index_id", &index_id) .replace("index_uri: #index_uri\n", ""), )?; + let index_config_with_retention_path = + resources_dir_path.join("index_config_with_retention.yaml"); + fs::write( + &index_config_with_retention_path, + format!("{DEFAULT_INDEX_CONFIG}{RETENTION_CONFIG}") + .replace("#index_id", &index_id) + .replace("#index_uri", index_uri.as_str()), + )?; let node_config_path = resources_dir_path.join("config.yaml"); let rest_listen_port = find_available_tcp_port()?; let grpc_listen_port = find_available_tcp_port()?; @@ -233,23 +261,18 @@ pub async fn create_test_env( let wikipedia_docs_path = resources_dir_path.join("wikis.json"); fs::write(&wikipedia_docs_path, WIKI_JSON_DOCS)?; - let mut resource_files = HashMap::new(); - resource_files.insert("config", node_config_path); - resource_files.insert("index_config", index_config_path); - resource_files.insert("index_config_without_uri", index_config_without_uri_path); - resource_files.insert("logs", log_docs_path); - resource_files.insert("wiki", wikipedia_docs_path); - - let config_uri = - Uri::from_str(&format!("file://{}", resource_files["config"].display())).unwrap(); - let index_config_uri = Uri::from_str(&format!( - "file://{}", - resource_files["index_config"].display() - )) - .unwrap(); let cluster_endpoint = Url::parse(&format!("http://localhost:{rest_listen_port}")) .context("failed to parse cluster endpoint")?; + let resource_files = TestResourceFiles { + config: uri_from_path(node_config_path), + index_config: uri_from_path(index_config_path), + index_config_without_uri: uri_from_path(index_config_without_uri_path), + index_config_with_retention: uri_from_path(index_config_with_retention_path), + log_docs: log_docs_path, + wikipedia_docs: wikipedia_docs_path, + }; + Ok(TestEnv { _temp_dir: temp_dir, data_dir_path, @@ -258,9 +281,7 @@ pub async fn create_test_env( metastore_uri, metastore_resolver, metastore, - config_uri, cluster_endpoint, - index_config_uri, index_id, index_uri, rest_listen_port, diff --git a/quickwit/quickwit-cluster/src/grpc_gossip.rs b/quickwit/quickwit-cluster/src/grpc_gossip.rs index 0b06ca5daa2..44b816b88e4 100644 --- a/quickwit/quickwit-cluster/src/grpc_gossip.rs +++ b/quickwit/quickwit-cluster/src/grpc_gossip.rs @@ -102,7 +102,7 @@ async fn perform_grpc_gossip_rounds( info!("pulling cluster state from node(s): {node_ids:?}"); for (node_id, grpc_advertise_addr) in zip(node_ids, grpc_advertise_addrs) { - let mut cluster_client = grpc_client_factory(grpc_advertise_addr).await; + let cluster_client = grpc_client_factory(grpc_advertise_addr).await; let request = FetchClusterStateRequest { cluster_id: cluster_id.clone(), diff --git a/quickwit/quickwit-cluster/src/grpc_service.rs b/quickwit/quickwit-cluster/src/grpc_service.rs index 4c0f3b63b53..a9c3ec4f7f2 100644 --- a/quickwit/quickwit-cluster/src/grpc_service.rs +++ b/quickwit/quickwit-cluster/src/grpc_service.rs @@ -60,7 +60,7 @@ pub fn cluster_grpc_server( #[async_trait] impl ClusterService for Cluster { async fn fetch_cluster_state( - &mut self, + &self, request: FetchClusterStateRequest, ) -> ClusterResult { if request.cluster_id != self.cluster_id() { @@ -132,7 +132,7 @@ mod tests { #[tokio::test] async fn test_fetch_cluster_state() { let transport = ChannelTransport::default(); - let mut cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) + let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); diff --git a/quickwit/quickwit-codegen/README.md b/quickwit/quickwit-codegen/README.md index 8553eb22b6a..0b5442dc49f 100644 --- a/quickwit/quickwit-codegen/README.md +++ b/quickwit/quickwit-codegen/README.md @@ -13,7 +13,6 @@ async-trait = { workspace = true } bytes = { workspace = true } bytesize = { workspace = true } -dyn-clone = { workspace = true } http = { workspace = true } hyper = { workspace = true } prost = { workspace = true } diff --git a/quickwit/quickwit-codegen/example/Cargo.toml b/quickwit/quickwit-codegen/example/Cargo.toml index 4e8189b80f0..8a178d2c957 100644 --- a/quickwit/quickwit-codegen/example/Cargo.toml +++ b/quickwit/quickwit-codegen/example/Cargo.toml @@ -14,7 +14,6 @@ license.workspace = true anyhow = { workspace = true } async-trait = { workspace = true } bytesize = { workspace = true } -dyn-clone = { workspace = true } futures = { workspace = true } http = { workspace = true } prost = { workspace = true } diff --git a/quickwit/quickwit-codegen/example/src/codegen/hello.rs b/quickwit/quickwit-codegen/example/src/codegen/hello.rs index ea48bb6224b..93b9b634ce7 100644 --- a/quickwit/quickwit-codegen/example/src/codegen/hello.rs +++ b/quickwit/quickwit-codegen/example/src/codegen/hello.rs @@ -63,36 +63,28 @@ impl RpcName for PingRequest { pub type HelloStream = quickwit_common::ServiceStream>; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait Hello: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait Hello: std::fmt::Debug + Send + Sync + 'static { /// Says hello. - async fn hello( - &mut self, - request: HelloRequest, - ) -> crate::HelloResult; + async fn hello(&self, request: HelloRequest) -> crate::HelloResult; /// Says goodbye. async fn goodbye( - &mut self, + &self, request: GoodbyeRequest, ) -> crate::HelloResult; /// Ping pong. async fn ping( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::HelloResult>; - async fn check_connectivity(&mut self) -> anyhow::Result<()>; + async fn check_connectivity(&self) -> anyhow::Result<()>; fn endpoints(&self) -> Vec; } -dyn_clone::clone_trait_object!(Hello); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockHello { - fn clone(&self) -> Self { - MockHello::new() - } -} #[derive(Debug, Clone)] pub struct HelloClient { - inner: Box, + inner: InnerHelloClient, } +#[derive(Debug, Clone)] +struct InnerHelloClient(std::sync::Arc); impl HelloClient { pub fn new(instance: T) -> Self where @@ -103,7 +95,9 @@ impl HelloClient { std::any::TypeId::of:: < T > () != std::any::TypeId::of:: < MockHello > (), "`MockHello` must be wrapped in a `MockHelloWrapper`: use `HelloClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerHelloClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -152,7 +146,7 @@ impl HelloClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockHello) -> Self { let mock_wrapper = mock_hello::MockHelloWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -163,59 +157,56 @@ impl HelloClient { } #[async_trait::async_trait] impl Hello for HelloClient { - async fn hello( - &mut self, - request: HelloRequest, - ) -> crate::HelloResult { - self.inner.hello(request).await + async fn hello(&self, request: HelloRequest) -> crate::HelloResult { + self.inner.0.hello(request).await } async fn goodbye( - &mut self, + &self, request: GoodbyeRequest, ) -> crate::HelloResult { - self.inner.goodbye(request).await + self.inner.0.goodbye(request).await } async fn ping( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::HelloResult> { - self.inner.ping(request).await + self.inner.0.ping(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { - self.inner.check_connectivity().await + async fn check_connectivity(&self) -> anyhow::Result<()> { + self.inner.0.check_connectivity().await } fn endpoints(&self) -> Vec { - self.inner.endpoints() + self.inner.0.endpoints() } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_hello { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockHelloWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl Hello for MockHelloWrapper { async fn hello( - &mut self, + &self, request: super::HelloRequest, ) -> crate::HelloResult { self.inner.lock().await.hello(request).await } async fn goodbye( - &mut self, + &self, request: super::GoodbyeRequest, ) -> crate::HelloResult { self.inner.lock().await.goodbye(request).await } async fn ping( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::HelloResult> { self.inner.lock().await.ping(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { self.inner.lock().await.check_connectivity().await } fn endpoints(&self) -> Vec { @@ -226,7 +217,7 @@ pub mod mock_hello { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerHelloClient { type Response = HelloResponse; type Error = crate::HelloError; type Future = BoxFuture; @@ -237,12 +228,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: HelloRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.hello(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.hello(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerHelloClient { type Response = GoodbyeResponse; type Error = crate::HelloError; type Future = BoxFuture; @@ -253,12 +244,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: GoodbyeRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.goodbye(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.goodbye(request).await }; Box::pin(fut) } } -impl tower::Service> for Box { +impl tower::Service> for InnerHelloClient { type Response = HelloStream; type Error = crate::HelloError; type Future = BoxFuture; @@ -272,15 +263,16 @@ impl tower::Service> for Box, ) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.ping(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.ping(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct HelloTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerHelloClient, hello_svc: quickwit_common::tower::BoxService< HelloRequest, HelloResponse, @@ -297,41 +289,28 @@ struct HelloTowerServiceStack { crate::HelloError, >, } -impl Clone for HelloTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - hello_svc: self.hello_svc.clone(), - goodbye_svc: self.goodbye_svc.clone(), - ping_svc: self.ping_svc.clone(), - } - } -} #[async_trait::async_trait] impl Hello for HelloTowerServiceStack { - async fn hello( - &mut self, - request: HelloRequest, - ) -> crate::HelloResult { - self.hello_svc.ready().await?.call(request).await + async fn hello(&self, request: HelloRequest) -> crate::HelloResult { + self.hello_svc.clone().ready().await?.call(request).await } async fn goodbye( - &mut self, + &self, request: GoodbyeRequest, ) -> crate::HelloResult { - self.goodbye_svc.ready().await?.call(request).await + self.goodbye_svc.clone().ready().await?.call(request).await } async fn ping( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::HelloResult> { - self.ping_svc.ready().await?.call(request).await + self.ping_svc.clone().ready().await?.call(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { - self.inner.check_connectivity().await + async fn check_connectivity(&self) -> anyhow::Result<()> { + self.inner.0.check_connectivity().await } fn endpoints(&self) -> Vec { - self.inner.endpoints() + self.inner.0.endpoints() } } type HelloLayer = quickwit_common::tower::BoxLayer< @@ -515,7 +494,8 @@ impl HelloTowerLayerStack { where T: Hello, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerHelloClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -523,20 +503,21 @@ impl HelloTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> HelloClient { - self.build_from_boxed( - Box::new(HelloClient::from_channel(addr, channel, max_message_size)), - ) + let client = HelloClient::from_channel(addr, channel, max_message_size); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> HelloClient { - self.build_from_boxed( - Box::new( - HelloClient::from_balance_channel(balance_channel, max_message_size), - ), - ) + let client = HelloClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -546,19 +527,24 @@ impl HelloTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, HelloMailbox: Hello, { - self.build_from_boxed(Box::new(HelloMailbox::new(mailbox))) + let inner_client = InnerHelloClient( + std::sync::Arc::new(HelloMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: MockHello) -> HelloClient { - self.build_from_boxed(Box::new(HelloClient::from_mock(mock))) + let client = HelloClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed(self, boxed_instance: Box) -> HelloClient { + fn build_from_inner_client(self, inner_client: InnerHelloClient) -> HelloClient { let hello_svc = self .hello_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let goodbye_svc = self @@ -566,7 +552,7 @@ impl HelloTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let ping_svc = self @@ -574,11 +560,11 @@ impl HelloTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = HelloTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, hello_svc, goodbye_svc, ping_svc, @@ -677,25 +663,22 @@ where Future = BoxFuture, crate::HelloError>, >, { - async fn hello( - &mut self, - request: HelloRequest, - ) -> crate::HelloResult { - self.call(request).await + async fn hello(&self, request: HelloRequest) -> crate::HelloResult { + self.clone().call(request).await } async fn goodbye( - &mut self, + &self, request: GoodbyeRequest, ) -> crate::HelloResult { - self.call(request).await + self.clone().call(request).await } async fn ping( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::HelloResult> { - self.call(request).await + self.clone().call(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { if self.inner.is_disconnected() { anyhow::bail!("actor `{}` is disconnected", self.inner.actor_instance_id()) } @@ -739,11 +722,9 @@ where + Send, T::Future: Send, { - async fn hello( - &mut self, - request: HelloRequest, - ) -> crate::HelloResult { + async fn hello(&self, request: HelloRequest) -> crate::HelloResult { self.inner + .clone() .hello(request) .await .map(|response| response.into_inner()) @@ -753,10 +734,11 @@ where )) } async fn goodbye( - &mut self, + &self, request: GoodbyeRequest, ) -> crate::HelloResult { self.inner + .clone() .goodbye(request) .await .map(|response| response.into_inner()) @@ -766,10 +748,11 @@ where )) } async fn ping( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::HelloResult> { self.inner + .clone() .ping(request) .await .map(|response| { @@ -786,7 +769,7 @@ where PingRequest::rpc_name(), )) } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { if self.connection_addrs_rx.borrow().len() == 0 { anyhow::bail!("no server currently available") } @@ -804,14 +787,16 @@ where } #[derive(Debug)] pub struct HelloGrpcServerAdapter { - inner: Box, + inner: InnerHelloClient, } impl HelloGrpcServerAdapter { pub fn new(instance: T) -> Self where T: Hello, { - Self { inner: Box::new(instance) } + Self { + inner: InnerHelloClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -821,7 +806,7 @@ impl hello_grpc_server::HelloGrpc for HelloGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .hello(request.into_inner()) .await .map(tonic::Response::new) @@ -832,7 +817,7 @@ impl hello_grpc_server::HelloGrpc for HelloGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .goodbye(request.into_inner()) .await .map(tonic::Response::new) @@ -844,7 +829,7 @@ impl hello_grpc_server::HelloGrpc for HelloGrpcServerAdapter { request: tonic::Request>, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .ping({ let streaming: tonic::Streaming<_> = request.into_inner(); quickwit_common::ServiceStream::from(streaming) diff --git a/quickwit/quickwit-codegen/example/src/lib.rs b/quickwit/quickwit-codegen/example/src/lib.rs index 1239e989b31..31572dafd94 100644 --- a/quickwit/quickwit-codegen/example/src/lib.rs +++ b/quickwit/quickwit-codegen/example/src/lib.rs @@ -120,7 +120,7 @@ struct HelloImpl { #[async_trait] impl Hello for HelloImpl { - async fn hello(&mut self, request: HelloRequest) -> HelloResult { + async fn hello(&self, request: HelloRequest) -> HelloResult { tokio::time::sleep(self.delay).await; if request.name.is_empty() { @@ -131,7 +131,7 @@ impl Hello for HelloImpl { }) } - async fn goodbye(&mut self, request: GoodbyeRequest) -> HelloResult { + async fn goodbye(&self, request: GoodbyeRequest) -> HelloResult { tokio::time::sleep(self.delay).await; Ok(GoodbyeResponse { @@ -140,13 +140,13 @@ impl Hello for HelloImpl { } async fn ping( - &mut self, + &self, request: ServiceStream, ) -> HelloResult> { Ok(spawn_ping_response_stream(request)) } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { Ok(()) } @@ -179,7 +179,7 @@ mod tests { #[tokio::test] async fn test_hello_codegen() { - let mut hello = HelloImpl::default(); + let hello = HelloImpl::default(); assert_eq!( hello @@ -193,7 +193,7 @@ mod tests { } ); - let mut client = HelloClient::new(hello.clone()).clone(); + let client = HelloClient::new(hello.clone()).clone(); assert_eq!( client @@ -283,7 +283,7 @@ mod tests { "127.0.0.1:6666".parse().unwrap(), Endpoint::from_static("http://127.0.0.1:6666").connect_lazy(), ); - let mut grpc_client = HelloClient::from_balance_channel(channel, MAX_GRPC_MESSAGE_SIZE); + let grpc_client = HelloClient::from_balance_channel(channel, MAX_GRPC_MESSAGE_SIZE); assert_eq!( grpc_client @@ -342,8 +342,7 @@ mod tests { // The connectivity check fails if there is no client behind the channel. let (balance_channel, _): (BalanceChannel, _) = BalanceChannel::new(); - let mut grpc_client = - HelloClient::from_balance_channel(balance_channel, MAX_GRPC_MESSAGE_SIZE); + let grpc_client = HelloClient::from_balance_channel(balance_channel, MAX_GRPC_MESSAGE_SIZE); assert_eq!( grpc_client .check_connectivity() @@ -411,7 +410,7 @@ mod tests { let universe = Universe::new(); let hello_actor = HelloActor; let (actor_mailbox, _actor_handle) = universe.spawn_builder().spawn(hello_actor); - let mut actor_client = HelloClient::from_mailbox(actor_mailbox.clone()); + let actor_client = HelloClient::from_mailbox(actor_mailbox.clone()); assert_eq!( actor_client @@ -448,7 +447,7 @@ mod tests { "Pong, beautiful actor!" ); - let mut hello_tower = HelloClient::tower().build_from_mailbox(actor_mailbox); + let hello_tower = HelloClient::tower().build_from_mailbox(actor_mailbox); assert_eq!( hello_tower @@ -499,7 +498,7 @@ mod tests { let goodbye_layer = CounterLayer::default(); let ping_layer = CounterLayer::default(); - let mut hello_tower = HelloClient::tower() + let hello_tower = HelloClient::tower() .stack_layer(layer.clone()) .stack_hello_layer(hello_layer.clone()) .stack_goodbye_layer(goodbye_layer.clone()) @@ -616,7 +615,7 @@ mod tests { } } } - let mut hello_tower = HelloClient::tower() + let hello_tower = HelloClient::tower() .stack_layer(AppendSuffixLayer::new("->foo")) .stack_hello_layer(AppendSuffixLayer::new("->bar")) .stack_layer(AppendSuffixLayer::new("->qux")) @@ -698,7 +697,7 @@ mod tests { }) }); mock_hello.expect_check_connectivity().returning(|| Ok(())); - let mut hello = HelloClient::from_mock(mock_hello); + let hello = HelloClient::from_mock(mock_hello); assert_eq!( hello @@ -735,7 +734,7 @@ mod tests { .timeout(Duration::from_millis(100)) .connect_lazy(); let max_message_size = ByteSize::mib(1); - let mut grpc_client = HelloClient::from_channel(addr, channel, max_message_size); + let grpc_client = HelloClient::from_channel(addr, channel, max_message_size); let error = grpc_client .hello(HelloRequest { diff --git a/quickwit/quickwit-codegen/src/codegen.rs b/quickwit/quickwit-codegen/src/codegen.rs index 71f4cb75fd2..72d5507af42 100644 --- a/quickwit/quickwit-codegen/src/codegen.rs +++ b/quickwit/quickwit-codegen/src/codegen.rs @@ -185,6 +185,7 @@ struct CodegenContext { stream_type_alias: TokenStream, methods: Vec, client_name: Ident, + inner_client_name: Ident, tower_svc_stack_name: Ident, tower_layer_stack_name: Ident, mailbox_name: Ident, @@ -227,6 +228,7 @@ impl CodegenContext { let methods = SynMethod::parse_prost_methods(&service.methods); let client_name = quote::format_ident!("{}Client", service.name); + let inner_client_name = quote::format_ident!("Inner{}", client_name); let tower_svc_stack_name = quote::format_ident!("{}TowerServiceStack", service.name); let tower_layer_stack_name = quote::format_ident!("{}TowerLayerStack", service.name); let mailbox_name = quote::format_ident!("{}Mailbox", service.name); @@ -253,6 +255,7 @@ impl CodegenContext { stream_type_alias, methods, client_name, + inner_client_name, tower_svc_stack_name, tower_layer_stack_name, mailbox_name, @@ -444,10 +447,9 @@ fn generate_comment_attributes(comments: &Comments) -> Vec { fn generate_service_trait(context: &CodegenContext) -> TokenStream { let service_name = &context.service_name; let trait_methods = generate_service_trait_methods(context); - let mock_name = &context.mock_name; let extra_trait_methods = if context.generate_extra_service_methods { quote! { - async fn check_connectivity(&mut self) -> anyhow::Result<()>; + async fn check_connectivity(&self) -> anyhow::Result<()>; fn endpoints(&self) -> Vec; } } else { @@ -457,19 +459,10 @@ fn generate_service_trait(context: &CodegenContext) -> TokenStream { quote! { #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] - pub trait #service_name: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { + pub trait #service_name: std::fmt::Debug + Send + Sync + 'static { #trait_methods #extra_trait_methods } - - dyn_clone::clone_trait_object!(#service_name); - - #[cfg(any(test, feature = "testsuite"))] - impl Clone for #mock_name { - fn clone(&self) -> Self { - #mock_name::new() - } - } } } @@ -485,7 +478,7 @@ fn generate_service_trait_methods(context: &CodegenContext) -> TokenStream { let response_type = syn_method.response_type(context, false); let method = quote! { #(#comments)* - async fn #method_name(&mut self, request: #request_type) -> #result_type<#response_type>; + async fn #method_name(&self, request: #request_type) -> #result_type<#response_type>; }; stream.extend(method); } @@ -494,12 +487,12 @@ fn generate_service_trait_methods(context: &CodegenContext) -> TokenStream { fn generate_extra_methods_calling_inner() -> TokenStream { quote! { - async fn check_connectivity(&mut self) -> anyhow::Result<()> { - self.inner.check_connectivity().await + async fn check_connectivity(&self) -> anyhow::Result<()> { + self.inner.0.check_connectivity().await } fn endpoints(&self) -> Vec { - self.inner.endpoints() + self.inner.0.endpoints() } } } @@ -507,6 +500,7 @@ fn generate_extra_methods_calling_inner() -> TokenStream { fn generate_client(context: &CodegenContext) -> TokenStream { let service_name = &context.service_name; let client_name = &context.client_name; + let inner_client_name = &context.inner_client_name; let grpc_client_name = &context.grpc_client_name; let grpc_client_adapter_name = &context.grpc_client_adapter_name; @@ -534,7 +528,7 @@ fn generate_client(context: &CodegenContext) -> TokenStream { }; let extra_mock_methods = if context.generate_extra_service_methods { quote! { - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { self.inner.lock().await.check_connectivity().await } @@ -549,9 +543,12 @@ fn generate_client(context: &CodegenContext) -> TokenStream { quote! { #[derive(Debug, Clone)] pub struct #client_name { - inner: Box + inner: #inner_client_name, } + #[derive(Debug, Clone)] + struct #inner_client_name(std::sync::Arc); + impl #client_name { pub fn new(instance: T) -> Self where @@ -560,7 +557,7 @@ fn generate_client(context: &CodegenContext) -> TokenStream { #[cfg(any(test, feature = "testsuite"))] assert!(std::any::TypeId::of::() != std::any::TypeId::of::<#mock_name>(), #error_mesage); Self { - inner: Box::new(instance), + inner: #inner_client_name(std::sync::Arc::new(instance)), } } @@ -606,7 +603,7 @@ fn generate_client(context: &CodegenContext) -> TokenStream { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: #mock_name) -> Self { let mock_wrapper = #mock_mod_name::#mock_wrapper_name { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)) + inner: tokio::sync::Mutex::new(mock) }; Self::new(mock_wrapper) } @@ -627,9 +624,9 @@ fn generate_client(context: &CodegenContext) -> TokenStream { pub mod #mock_mod_name { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct #mock_wrapper_name { - pub(super) inner: std::sync::Arc> + pub(super) inner: tokio::sync::Mutex<#mock_name> } #[async_trait::async_trait] @@ -653,7 +650,7 @@ fn generate_client_methods(context: &CodegenContext, mock: bool) -> TokenStream let body = if !mock { quote! { - self.inner.#method_name(request).await + self.inner.0.#method_name(request).await } } else { quote! { @@ -661,7 +658,7 @@ fn generate_client_methods(context: &CodegenContext, mock: bool) -> TokenStream } }; let method = quote! { - async fn #method_name(&mut self, request: #request_type) -> #result_type<#response_type> { + async fn #method_name(&self, request: #request_type) -> #result_type<#response_type> { #body } }; @@ -671,7 +668,7 @@ fn generate_client_methods(context: &CodegenContext, mock: bool) -> TokenStream } fn generate_tower_services(context: &CodegenContext) -> TokenStream { - let service_name = &context.service_name; + let inner_client_name = &context.inner_client_name; let error_type = &context.error_type; let mut stream = TokenStream::new(); @@ -682,7 +679,7 @@ fn generate_tower_services(context: &CodegenContext) -> TokenStream { let response_type = syn_method.response_type(context, false); let service = quote! { - impl tower::Service<#request_type> for Box { + impl tower::Service<#request_type> for #inner_client_name { type Response = #response_type; type Error = #error_type; type Future = BoxFuture; @@ -695,8 +692,8 @@ fn generate_tower_services(context: &CodegenContext) -> TokenStream { } fn call(&mut self, request: #request_type) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.#method_name(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.#method_name(request).await }; Box::pin(fut) } } @@ -708,22 +705,21 @@ fn generate_tower_services(context: &CodegenContext) -> TokenStream { fn generate_tower_svc_stack(context: &CodegenContext) -> TokenStream { let tower_svc_stack_name = &context.tower_svc_stack_name; - let service_name = &context.service_name; + let inner_client_name = &context.inner_client_name; let tower_svc_stack_attributes = generate_tower_svc_stack_attributes(context); - let tower_svc_stack_clone_impl = generate_tower_svc_stack_clone_impl(context); let tower_svc_stack_service_impl = generate_tower_svc_stack_service_impl(context); quote! { /// A tower service stack is a set of tower services. #[derive(Debug)] struct #tower_svc_stack_name { - inner: Box, + // TODO: remove this field once `check_connectivity` is used for all services. + #[allow(dead_code)] + inner: #inner_client_name, #tower_svc_stack_attributes } - #tower_svc_stack_clone_impl - #tower_svc_stack_service_impl } } @@ -746,31 +742,6 @@ fn generate_tower_svc_stack_attributes(context: &CodegenContext) -> TokenStream stream } -fn generate_tower_svc_stack_clone_impl(context: &CodegenContext) -> TokenStream { - let tower_svc_stack_name = &context.tower_svc_stack_name; - - let mut cloned_attributes = TokenStream::new(); - - for syn_method in &context.methods { - let attribute_name = quote::format_ident!("{}_svc", syn_method.name); - let attribute = quote! { - #attribute_name: self.#attribute_name.clone(), - }; - cloned_attributes.extend(attribute); - } - - quote! { - impl Clone for #tower_svc_stack_name { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - #cloned_attributes - } - } - } - } -} - fn generate_tower_svc_stack_service_impl(context: &CodegenContext) -> TokenStream { let service_name = &context.service_name; let tower_svc_stack_name = &context.tower_svc_stack_name; @@ -789,8 +760,8 @@ fn generate_tower_svc_stack_service_impl(context: &CodegenContext) -> TokenStrea let response_type = syn_method.response_type(context, false); let attribute = quote! { - async fn #method_name(&mut self, request: #request_type) -> #result_type<#response_type> { - self.#attribute_name.ready().await?.call(request).await + async fn #method_name(&self, request: #request_type) -> #result_type<#response_type> { + self.#attribute_name.clone().ready().await?.call(request).await } }; methods.extend(attribute); @@ -853,6 +824,7 @@ fn generate_layer_stack_types_and_attributes( fn generate_layer_stack_impl(context: &CodegenContext) -> TokenStream { let service_name = &context.service_name; let client_name = &context.client_name; + let inner_client_name = &context.inner_client_name; let mailbox_name = &context.mailbox_name; let mock_name = &context.mock_name; let tower_svc_stack_name = &context.tower_svc_stack_name; @@ -904,7 +876,7 @@ fn generate_layer_stack_impl(context: &CodegenContext) -> TokenStream { layer_methods.extend(layer_method); let svc_statement = quote! { - let #svc_attribute_name = self.#layer_attribute_name.into_iter().rev().fold(quickwit_common::tower::BoxService::new(boxed_instance.clone()), |svc, layer| layer.layer(svc)); + let #svc_attribute_name = self.#layer_attribute_name.into_iter().rev().fold(quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc)); }; svc_statements.extend(svc_statement); @@ -927,17 +899,22 @@ fn generate_layer_stack_impl(context: &CodegenContext) -> TokenStream { where T: #service_name { - self.build_from_boxed(Box::new(instance)) + let inner_client = #inner_client_name(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel(self, addr: std::net::SocketAddr, channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize) -> #client_name { - self.build_from_boxed(Box::new(#client_name::from_channel(addr, channel, max_message_size))) + let client = #client_name::from_channel(addr, channel, max_message_size); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel(self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize) -> #client_name { - self.build_from_boxed(Box::new(#client_name::from_balance_channel(balance_channel, max_message_size))) + let client = #client_name::from_balance_channel(balance_channel, max_message_size); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox(self, mailbox: quickwit_actors::Mailbox) -> #client_name @@ -945,20 +922,23 @@ fn generate_layer_stack_impl(context: &CodegenContext) -> TokenStream { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, #mailbox_name: #service_name, { - self.build_from_boxed(Box::new(#mailbox_name::new(mailbox))) + let inner_client = #inner_client_name(std::sync::Arc::new(#mailbox_name::new(mailbox))); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: #mock_name) -> #client_name { - self.build_from_boxed(Box::new(#client_name::from_mock(mock))) + let client = #client_name::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed(self, boxed_instance: Box) -> #client_name + fn build_from_inner_client(self, inner_client: #inner_client_name) -> #client_name { #svc_statements let tower_svc_stack = #tower_svc_stack_name { - inner: boxed_instance.clone(), + inner: inner_client, #(#svc_attribute_idents),* }; #client_name::new(tower_svc_stack) @@ -973,7 +953,7 @@ fn generate_tower_mailbox(context: &CodegenContext) -> TokenStream { let error_type = &context.error_type; let extra_mailbox_methods = if context.generate_extra_service_methods { quote! { - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { if self.inner.is_disconnected() { anyhow::bail!("actor `{}` is disconnected", self.inner.actor_instance_id()) } @@ -1095,8 +1075,8 @@ fn generate_mailbox_bounds_and_methods( bounds.push(bound); let method = quote! { - async fn #method_name(&mut self, request: #request_type) -> #result_type<#response_type> { - self.call(request).await + async fn #method_name(&self, request: #request_type) -> #result_type<#response_type> { + self.clone().call(request).await } }; methods.extend(method); @@ -1114,7 +1094,7 @@ fn generate_grpc_client_adapter(context: &CodegenContext) -> TokenStream { let grpc_server_adapter_methods = generate_grpc_client_adapter_methods(context); let extra_grpc_server_adapter_methods = if context.generate_extra_service_methods { quote! { - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { if self.connection_addrs_rx.borrow().len() == 0 { anyhow::bail!("no server currently available") } @@ -1188,8 +1168,9 @@ fn generate_grpc_client_adapter_methods(context: &CodegenContext) -> TokenStream quote! { |response| response.into_inner() } }; let method = quote! { - async fn #method_name(&mut self, request: #request_type) -> #result_type<#response_type> { + async fn #method_name(&self, request: #request_type) -> #result_type<#response_type> { self.inner + .clone() .#method_name(request) .await .map(#into_response_type) @@ -1203,6 +1184,7 @@ fn generate_grpc_client_adapter_methods(context: &CodegenContext) -> TokenStream fn generate_grpc_server_adapter(context: &CodegenContext) -> TokenStream { let service_name = &context.service_name; + let inner_client_name = &context.inner_client_name; let grpc_server_package_name = &context.grpc_server_package_name; let grpc_service_name = &context.grpc_service_name; let grpc_server_adapter_name = &context.grpc_server_adapter_name; @@ -1211,14 +1193,14 @@ fn generate_grpc_server_adapter(context: &CodegenContext) -> TokenStream { quote! { #[derive(Debug)] pub struct #grpc_server_adapter_name { - inner: Box, + inner: #inner_client_name, } impl #grpc_server_adapter_name { pub fn new(instance: T) -> Self where T: #service_name { Self { - inner: Box::new(instance), + inner: #inner_client_name(std::sync::Arc::new(instance)), } } } @@ -1276,7 +1258,7 @@ fn generate_grpc_server_adapter_methods(context: &CodegenContext) -> TokenStream async fn #method_name(&self, request: tonic::Request<#request_type>) -> Result, tonic::Status> { self.inner - .clone() + .0 .#method_name(#method_arg) .await .map(#into_response_type) diff --git a/quickwit/quickwit-config/src/index_config/mod.rs b/quickwit/quickwit-config/src/index_config/mod.rs index 574c68dea45..48adf301b75 100644 --- a/quickwit/quickwit-config/src/index_config/mod.rs +++ b/quickwit/quickwit-config/src/index_config/mod.rs @@ -34,7 +34,7 @@ use quickwit_common::uri::Uri; use quickwit_doc_mapper::{DefaultDocMapperBuilder, DocMapper, DocMapping, Mode}; use quickwit_proto::types::IndexId; use serde::{Deserialize, Serialize}; -pub use serialize::load_index_config_from_user_config; +pub use serialize::{load_index_config_from_user_config, load_index_config_update}; use tracing::warn; use crate::index_config::serialize::VersionedIndexConfig; diff --git a/quickwit/quickwit-config/src/index_config/serialize.rs b/quickwit/quickwit-config/src/index_config/serialize.rs index 07885867962..b02322d5339 100644 --- a/quickwit/quickwit-config/src/index_config/serialize.rs +++ b/quickwit/quickwit-config/src/index_config/serialize.rs @@ -17,7 +17,7 @@ // You should have received a copy of the GNU Affero General Public License // along with this program. If not, see . -use anyhow::Context; +use anyhow::{ensure, Context}; use quickwit_common::uri::Uri; use quickwit_proto::types::IndexId; use serde::{Deserialize, Serialize}; @@ -61,6 +61,45 @@ pub fn load_index_config_from_user_config( index_config_for_serialization.build_and_validate(Some(default_index_root_uri)) } +/// Parses and validates an [`IndexConfig`] update. +/// +/// Ensures that the new configuration is valid in itself and compared to the +/// current index config. If the new configuration omits some fields, the +/// default values will be used, not those of the current index config. The only +/// exception is the index_uri because it cannot be updated. +pub fn load_index_config_update( + config_format: ConfigFormat, + index_config_bytes: &[u8], + current_index_config: &IndexConfig, +) -> anyhow::Result { + let current_index_parent_dir = ¤t_index_config + .index_uri + .parent() + .context("Unexpected `index_uri` format on current configuration")?; + let new_index_config = load_index_config_from_user_config( + config_format, + index_config_bytes, + current_index_parent_dir, + )?; + ensure!( + current_index_config.index_id == new_index_config.index_id, + "`index_id` in config file {} does not match updated `index_id` {}", + current_index_config.index_id, + new_index_config.index_id + ); + ensure!( + current_index_config.index_uri == new_index_config.index_uri, + "`index_uri` cannot be updated, current value {}, new expected value {}", + current_index_config.index_uri, + new_index_config.index_uri + ); + ensure!( + current_index_config.doc_mapping == new_index_config.doc_mapping, + "`doc_mapping` cannot be updated" + ); + Ok(new_index_config) +} + impl IndexConfigForSerialization { fn index_uri_or_fallback_to_default( &self, @@ -246,4 +285,155 @@ mod test { assert_eq!(index_config.index_uri.as_str(), "s3://mybucket/hdfs-logs"); } } + + #[test] + fn test_update_index_root_uri() { + let original_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + doc_mapping: {} + "#; + let original_config: IndexConfig = load_index_config_from_user_config( + ConfigFormat::Yaml, + original_config_yaml.as_bytes(), + &Uri::for_test("s3://mybucket"), + ) + .unwrap(); + { + // use default in update + let updated_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + doc_mapping: {} + "#; + let updated_config = load_index_config_update( + ConfigFormat::Yaml, + updated_config_yaml.as_bytes(), + &original_config, + ) + .unwrap(); + assert_eq!(updated_config.index_uri.as_str(), "s3://mybucket/hdfs-logs"); + } + { + // use the current index_uri explicitely + let updated_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + index_uri: s3://mybucket/hdfs-logs + doc_mapping: {} + "#; + let updated_config = load_index_config_update( + ConfigFormat::Yaml, + updated_config_yaml.as_bytes(), + &original_config, + ) + .unwrap(); + assert_eq!(updated_config.index_uri.as_str(), "s3://mybucket/hdfs-logs"); + } + { + // try using a different index_uri + let updated_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + index_uri: s3://mybucket/new-directory/ + doc_mapping: {} + "#; + let load_error = load_index_config_update( + ConfigFormat::Yaml, + updated_config_yaml.as_bytes(), + &original_config, + ) + .unwrap_err(); + assert!(format!("{:?}", load_error).contains("`index_uri` cannot be updated")); + } + } + + #[test] + fn test_update_reset_defaults() { + let original_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + doc_mapping: + field_mappings: + - name: timestamp + type: datetime + fast: true + timestamp_field: timestamp + + search_settings: + default_search_fields: [body] + + indexing_settings: + commit_timeout_secs: 10 + + retention: + period: 90 days + schedule: daily + "#; + let original_config: IndexConfig = load_index_config_from_user_config( + ConfigFormat::Yaml, + original_config_yaml.as_bytes(), + &Uri::for_test("s3://mybucket"), + ) + .unwrap(); + + let updated_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + doc_mapping: + field_mappings: + - name: timestamp + type: datetime + fast: true + timestamp_field: timestamp + "#; + let updated_config = load_index_config_update( + ConfigFormat::Yaml, + updated_config_yaml.as_bytes(), + &original_config, + ) + .unwrap(); + assert_eq!( + updated_config.search_settings.default_search_fields, + Vec::::default(), + ); + assert_eq!( + updated_config.indexing_settings.commit_timeout_secs, + IndexingSettings::default_commit_timeout_secs() + ); + assert_eq!(updated_config.retention_policy_opt, None); + } + + #[test] + fn test_update_doc_mappings() { + let original_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + doc_mapping: {} + "#; + let original_config: IndexConfig = load_index_config_from_user_config( + ConfigFormat::Yaml, + original_config_yaml.as_bytes(), + &Uri::for_test("s3://mybucket"), + ) + .unwrap(); + + let updated_config_yaml = r#" + version: 0.8 + index_id: hdfs-logs + doc_mapping: + field_mappings: + - name: body + type: text + tokenizer: default + record: position + "#; + let load_error = load_index_config_update( + ConfigFormat::Yaml, + updated_config_yaml.as_bytes(), + &original_config, + ) + .unwrap_err(); + assert!(format!("{:?}", load_error).contains("`doc_mapping` cannot be updated")); + } } diff --git a/quickwit/quickwit-config/src/lib.rs b/quickwit/quickwit-config/src/lib.rs index 9a3f9263aa1..e41c46dce5b 100644 --- a/quickwit/quickwit-config/src/lib.rs +++ b/quickwit/quickwit-config/src/lib.rs @@ -48,8 +48,8 @@ pub use cluster_config::ClusterConfig; // See #2048 use index_config::serialize::{IndexConfigV0_8, VersionedIndexConfig}; pub use index_config::{ - build_doc_mapper, load_index_config_from_user_config, IndexConfig, IndexingResources, - IndexingSettings, RetentionPolicy, SearchSettings, + build_doc_mapper, load_index_config_from_user_config, load_index_config_update, IndexConfig, + IndexingResources, IndexingSettings, RetentionPolicy, SearchSettings, }; pub use quickwit_doc_mapper::DocMapping; use serde::de::DeserializeOwned; diff --git a/quickwit/quickwit-control-plane/src/control_plane.rs b/quickwit/quickwit-control-plane/src/control_plane.rs index 33c9aa0dfcd..0f70d3b5ae2 100644 --- a/quickwit/quickwit-control-plane/src/control_plane.rs +++ b/quickwit/quickwit-control-plane/src/control_plane.rs @@ -39,7 +39,7 @@ use quickwit_common::Progress; use quickwit_config::service::QuickwitService; use quickwit_config::{ClusterConfig, IndexConfig, IndexTemplate, SourceConfig}; use quickwit_ingest::{IngesterPool, LocalShardsUpdate}; -use quickwit_metastore::{CreateIndexRequestExt, CreateIndexResponseExt}; +use quickwit_metastore::{CreateIndexRequestExt, CreateIndexResponseExt, IndexMetadataResponseExt}; use quickwit_proto::control_plane::{ AdviseResetShardsRequest, AdviseResetShardsResponse, ControlPlaneError, ControlPlaneResult, GetOrCreateOpenShardsRequest, GetOrCreateOpenShardsResponse, GetOrCreateOpenShardsSubrequest, @@ -280,7 +280,7 @@ impl ControlPlane { &source_configs, )?; let create_index_future = { - let mut metastore = self.metastore.clone(); + let metastore = self.metastore.clone(); async move { metastore.create_index(create_index_request).await } }; create_index_futures.push(create_index_future); @@ -582,6 +582,15 @@ impl Handler for ControlPlane { return convert_metastore_error(metastore_error); } }; + let index_metadata = match response.deserialize_index_metadata() { + Ok(index_metadata) => index_metadata, + Err(serde_error) => { + error!(error=?serde_error, "failed to deserialize index metadata"); + return Err(ActorExitStatus::from(anyhow::anyhow!(serde_error))); + } + }; + self.model + .update_index_config(&index_uid, index_metadata.index_config)?; // TODO: Handle doc mapping and/or indexing settings update here. info!(%index_uid, "updated index"); Ok(Ok(response)) diff --git a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs index 4c7fe5522bb..6084d30db86 100644 --- a/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs +++ b/quickwit/quickwit-control-plane/src/ingest/ingest_controller.rs @@ -226,7 +226,7 @@ impl IngestController { fn sync_with_ingester(&self, ingester: &NodeId, model: &ControlPlaneModel) -> WaitHandle { info!(ingester = %ingester, "sync_with_ingester"); let (wait_drop_guard, wait_handle) = WaitHandle::new(); - let Some(mut ingester_client) = self.ingester_pool.get(ingester) else { + let Some(ingester_client) = self.ingester_pool.get(ingester) else { // TODO: (Maybe) We should mark the ingester as unavailable, and stop advertise its // shard to routers. warn!("failed to sync with ingester `{ingester}`: not available"); @@ -535,7 +535,7 @@ impl IngestController { } }) .collect(); - let Some(mut leader) = self.ingester_pool.get(&leader_id) else { + let Some(leader) = self.ingester_pool.get(&leader_id) else { warn!("failed to init shards: ingester `{leader_id}` is unavailable"); failures.extend(init_shard_failures); continue; @@ -765,7 +765,7 @@ impl IngestController { model.release_scaling_permits(&source_uid, ScalingMode::Down, NUM_PERMITS); return Ok(()); }; - let Some(mut ingester) = self.ingester_pool.get(&leader_id) else { + let Some(ingester) = self.ingester_pool.get(&leader_id) else { model.release_scaling_permits(&source_uid, ScalingMode::Down, NUM_PERMITS); return Ok(()); }; @@ -1024,7 +1024,7 @@ impl IngestController { let mut close_shards_futures = FuturesUnordered::new(); for (leader_id, shard_pkeys) in per_leader_shards_to_close { - let Some(mut ingester) = self.ingester_pool.get(&leader_id) else { + let Some(ingester) = self.ingester_pool.get(&leader_id) else { warn!("failed to close shards: ingester `{leader_id}` is unavailable"); continue; }; diff --git a/quickwit/quickwit-control-plane/src/model/mod.rs b/quickwit/quickwit-control-plane/src/model/mod.rs index 229f9275558..8948eb0cfca 100644 --- a/quickwit/quickwit-control-plane/src/model/mod.rs +++ b/quickwit/quickwit-control-plane/src/model/mod.rs @@ -29,7 +29,7 @@ use anyhow::bail; use fnv::{FnvHashMap, FnvHashSet}; use quickwit_common::pretty::PrettyDisplay; use quickwit_common::Progress; -use quickwit_config::SourceConfig; +use quickwit_config::{IndexConfig, SourceConfig}; use quickwit_ingest::ShardInfos; use quickwit_metastore::{IndexMetadata, ListIndexesMetadataResponseExt}; use quickwit_proto::control_plane::ControlPlaneResult; @@ -194,6 +194,21 @@ impl ControlPlaneModel { self.update_metrics(); } + /// Updates the configuration of the specified index, returning an error if + /// the index didn't exist. + pub(crate) fn update_index_config( + &mut self, + index_uid: &IndexUid, + index_config: IndexConfig, + ) -> anyhow::Result<()> { + let Some(index_model) = self.index_table.get_mut(index_uid) else { + bail!("index `{}` not found", index_uid.index_id); + }; + index_model.index_config = index_config; + self.update_metrics(); + Ok(()) + } + pub(crate) fn delete_index(&mut self, index_uid: &IndexUid) { self.index_table.remove(index_uid); self.index_uid_table.remove(&index_uid.index_id); @@ -529,6 +544,27 @@ mod tests { assert_eq!(model.shard_table.get_shards(&source_uid).unwrap().len(), 0); } + #[test] + fn test_control_plane_model_update_index_config() { + let mut model = ControlPlaneModel::default(); + let index_metadata = IndexMetadata::for_test("test-index", "ram:///indexes"); + let index_uid = index_metadata.index_uid.clone(); + model.add_index(index_metadata.clone()); + + // Update the index config + let mut index_config = index_metadata.index_config.clone(); + index_config.search_settings.default_search_fields = vec!["myfield".to_string()]; + model + .update_index_config(&index_uid, index_config.clone()) + .unwrap(); + + assert_eq!(model.index_table.len(), 1); + assert_eq!( + model.index_table.get(&index_uid).unwrap().index_config, + index_config + ); + } + #[test] fn test_control_plane_model_delete_index() { let mut model = ControlPlaneModel::default(); diff --git a/quickwit/quickwit-index-management/src/garbage_collection.rs b/quickwit/quickwit-index-management/src/garbage_collection.rs index 268219407b3..2a2d0e63581 100644 --- a/quickwit/quickwit-index-management/src/garbage_collection.rs +++ b/quickwit/quickwit-index-management/src/garbage_collection.rs @@ -87,7 +87,7 @@ pub struct SplitRemovalInfo { pub async fn run_garbage_collect( index_uid: IndexUid, storage: Arc, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, staged_grace_period: Duration, deletion_grace_period: Duration, dry_run: bool, @@ -175,7 +175,7 @@ async fn delete_splits_marked_for_deletion( index_uid: IndexUid, updated_before_timestamp: i64, storage: Arc, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, progress_opt: Option<&Progress>, ) -> SplitRemovalInfo { let mut removed_splits = Vec::new(); @@ -257,7 +257,7 @@ async fn delete_splits_marked_for_deletion( pub async fn delete_splits_from_storage_and_metastore( index_uid: IndexUid, storage: Arc, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, splits: Vec, progress_opt: Option<&Progress>, ) -> anyhow::Result, DeleteSplitsError> { @@ -372,7 +372,7 @@ mod tests { #[tokio::test] async fn test_run_gc_marks_stale_staged_splits_for_deletion_after_grace_period() { let storage = storage_for_test(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = "test-run-gc--index"; let index_uri = format!("ram:///indexes/{index_id}"); @@ -472,7 +472,7 @@ mod tests { #[tokio::test] async fn test_run_gc_deletes_splits_marked_for_deletion_after_grace_period() { let storage = storage_for_test(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = "test-run-gc--index"; let index_uri = format!("ram:///indexes/{index_id}"); @@ -599,7 +599,7 @@ mod tests { #[tokio::test] async fn test_delete_splits_from_storage_and_metastore_happy_path() { let storage = storage_for_test(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = "test-delete-splits-happy--index"; let index_uri = format!("ram:///indexes/{index_id}"); @@ -702,7 +702,7 @@ mod tests { Err(bulk_delete_error) }); let storage = Arc::new(mock_storage); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = "test-delete-splits-storage-error--index"; let index_uri = format!("ram:///indexes/{index_id}"); diff --git a/quickwit/quickwit-index-management/src/index.rs b/quickwit/quickwit-index-management/src/index.rs index 0d87d6bde76..51377720594 100644 --- a/quickwit/quickwit-index-management/src/index.rs +++ b/quickwit/quickwit-index-management/src/index.rs @@ -125,7 +125,7 @@ impl IndexService { } } } - let mut metastore = self.metastore.clone(); + let metastore = self.metastore.clone(); let index_config_json = serde_utils::to_json_str(&index_config)?; @@ -253,7 +253,7 @@ impl IndexService { } } - let mut metastore = self.metastore.clone(); + let metastore = self.metastore.clone(); let indexes_metadata = metastore .list_indexes_metadata(list_indexes_metadatas_request) .await? @@ -525,7 +525,7 @@ mod tests { #[tokio::test] async fn test_create_index() { - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let storage_resolver = StorageResolver::for_test(); let mut index_service = IndexService::new(metastore.clone(), storage_resolver); let index_id = "test-index"; diff --git a/quickwit/quickwit-indexing/failpoints/mod.rs b/quickwit/quickwit-indexing/failpoints/mod.rs index 0e468bc20ea..70d7aeacabd 100644 --- a/quickwit/quickwit-indexing/failpoints/mod.rs +++ b/quickwit/quickwit-indexing/failpoints/mod.rs @@ -268,7 +268,7 @@ async fn test_merge_executor_controlled_directory_kill_switch() -> anyhow::Resul } tokio::time::sleep(Duration::from_millis(10)).await; - let mut metastore = test_index_builder.metastore(); + let metastore = test_index_builder.metastore(); let split_metadatas: Vec = metastore .list_splits(ListSplitsRequest::try_from_index_uid(test_index_builder.index_uid()).unwrap()) .await? diff --git a/quickwit/quickwit-indexing/src/actors/indexing_service.rs b/quickwit/quickwit-indexing/src/actors/indexing_service.rs index 18d234be3d3..aa8d616bd35 100644 --- a/quickwit/quickwit-indexing/src/actors/indexing_service.rs +++ b/quickwit/quickwit-indexing/src/actors/indexing_service.rs @@ -1037,7 +1037,7 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); @@ -1139,7 +1139,7 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); @@ -1196,7 +1196,7 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); @@ -1426,7 +1426,7 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_id = append_random_suffix("test-indexing-service"); let index_uri = format!("ram:///indexes/{index_id}"); @@ -1635,7 +1635,7 @@ mod tests { let cluster = create_cluster_for_test(Vec::new(), &["indexer"], &transport, true) .await .unwrap(); - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let create_index_request = CreateIndexRequest::try_from_index_config(&index_config).unwrap(); let index_uid: IndexUid = metastore diff --git a/quickwit/quickwit-indexing/src/actors/merge_executor.rs b/quickwit/quickwit-indexing/src/actors/merge_executor.rs index 64b59cb2aa1..3324bd90754 100644 --- a/quickwit/quickwit-indexing/src/actors/merge_executor.rs +++ b/quickwit/quickwit-indexing/src/actors/merge_executor.rs @@ -591,7 +591,7 @@ mod tests { ); test_sandbox.add_documents(single_doc).await?; } - let mut metastore = test_sandbox.metastore(); + let metastore = test_sandbox.metastore(); let index_uid = test_sandbox.index_uid(); let list_splits_request = ListSplitsRequest::try_from_index_uid(index_uid.clone()).unwrap(); let split_metas: Vec = metastore @@ -709,7 +709,7 @@ mod tests { "#; let test_sandbox = TestSandbox::create(index_id, doc_mapping_yaml, "", &["body"]).await?; test_sandbox.add_documents(docs).await?; - let mut metastore = test_sandbox.metastore(); + let metastore = test_sandbox.metastore(); let index_uid = test_sandbox.index_uid(); metastore .create_delete_task(DeleteQuery { @@ -832,7 +832,7 @@ mod tests { } } else { assert!(packager_msgs.is_empty()); - let mut metastore = test_sandbox.metastore(); + let metastore = test_sandbox.metastore(); let index_uid = test_sandbox.index_uid(); let splits = metastore .list_splits(ListSplitsRequest::try_from_index_uid(index_uid).unwrap()) diff --git a/quickwit/quickwit-indexing/src/source/ingest/mod.rs b/quickwit/quickwit-indexing/src/source/ingest/mod.rs index 692cc0a02d0..3d4b85190d4 100644 --- a/quickwit/quickwit-indexing/src/source/ingest/mod.rs +++ b/quickwit/quickwit-indexing/src/source/ingest/mod.rs @@ -350,7 +350,7 @@ impl IngestSource { .push(truncate_shards_subrequest); } for (ingester_id, truncate_subrequests) in per_ingester_truncate_subrequests { - let Some(mut ingester) = self.ingester_pool.get(ingester_id) else { + let Some(ingester) = self.ingester_pool.get(ingester_id) else { warn!("failed to truncate shard(s): ingester `{ingester_id}` is unavailable"); continue; }; diff --git a/quickwit/quickwit-indexing/src/source/kafka_source.rs b/quickwit/quickwit-indexing/src/source/kafka_source.rs index b18c1ef9e83..4498cb65c84 100644 --- a/quickwit/quickwit-indexing/src/source/kafka_source.rs +++ b/quickwit/quickwit-indexing/src/source/kafka_source.rs @@ -914,7 +914,7 @@ mod kafka_broker_tests { } async fn setup_index( - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, index_id: &str, source_config: &SourceConfig, partition_deltas: &[(u64, i64, i64)], diff --git a/quickwit/quickwit-indexing/src/source/pulsar_source.rs b/quickwit/quickwit-indexing/src/source/pulsar_source.rs index 2e19ddc833d..6dec087a454 100644 --- a/quickwit/quickwit-indexing/src/source/pulsar_source.rs +++ b/quickwit/quickwit-indexing/src/source/pulsar_source.rs @@ -493,7 +493,7 @@ mod pulsar_broker_tests { } async fn setup_index( - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, index_id: &str, source_id: &str, partition_deltas: &[(&str, Position, Position)], diff --git a/quickwit/quickwit-indexing/src/test_utils.rs b/quickwit/quickwit-indexing/src/test_utils.rs index 36dff368304..3e6623be79b 100644 --- a/quickwit/quickwit-indexing/src/test_utils.rs +++ b/quickwit/quickwit-indexing/src/test_utils.rs @@ -96,7 +96,7 @@ impl TestSandbox { let storage_resolver = StorageResolver::for_test(); let metastore_resolver = MetastoreResolver::configured(storage_resolver.clone(), &MetastoreConfigs::default()); - let mut metastore = metastore_resolver + let metastore = metastore_resolver .resolve(&Uri::for_test(METASTORE_URI)) .await?; let create_index_request = CreateIndexRequest::try_from_index_and_source_configs( @@ -321,7 +321,7 @@ mod tests { serde_json::json!({"title": "Ganimede", "body": "...", "url": "http://ganimede"}), ]).await?; assert_eq!(statistics.num_uploaded_splits, 1); - let mut metastore = test_sandbox.metastore(); + let metastore = test_sandbox.metastore(); { let splits = metastore .list_splits( diff --git a/quickwit/quickwit-ingest/Cargo.toml b/quickwit/quickwit-ingest/Cargo.toml index 80159980ea2..0e0860028d0 100644 --- a/quickwit/quickwit-ingest/Cargo.toml +++ b/quickwit/quickwit-ingest/Cargo.toml @@ -15,7 +15,6 @@ anyhow = { workspace = true } async-trait = { workspace = true } bytes = { workspace = true } bytesize = { workspace = true } -dyn-clone = { workspace = true } fail = { workspace = true, optional = true } futures = { workspace = true } http = { workspace = true } diff --git a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs index 8ee2723daa5..ac03fd52faf 100644 --- a/quickwit/quickwit-ingest/src/codegen/ingest_service.rs +++ b/quickwit/quickwit-ingest/src/codegen/ingest_service.rs @@ -182,7 +182,7 @@ impl RpcName for TailRequest { } #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait IngestService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait IngestService: std::fmt::Debug + Send + Sync + 'static { /// Ingests document in a given queue. /// /// Upon any kind of error, the client should @@ -190,7 +190,7 @@ pub trait IngestService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + ' /// - not retry to get at most once delivery. /// /// Exactly once delivery is not supported yet. - async fn ingest(&mut self, request: IngestRequest) -> crate::Result; + async fn ingest(&self, request: IngestRequest) -> crate::Result; /// Fetches record from a given queue. /// /// Records are returned in order. @@ -201,25 +201,20 @@ pub trait IngestService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + ' /// Fetching does not necessarily return all of the available records. /// If returning all records would exceed `FETCH_PAYLOAD_LIMIT` (2MB), /// the response will be partial. - async fn fetch(&mut self, request: FetchRequest) -> crate::Result; + async fn fetch(&self, request: FetchRequest) -> crate::Result; /// Returns a batch containing the last records. /// /// It returns the last documents, from the newest /// to the oldest, and stops as soon as `FETCH_PAYLOAD_LIMIT` (2MB) /// is exceeded. - async fn tail(&mut self, request: TailRequest) -> crate::Result; -} -dyn_clone::clone_trait_object!(IngestService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockIngestService { - fn clone(&self) -> Self { - MockIngestService::new() - } + async fn tail(&self, request: TailRequest) -> crate::Result; } #[derive(Debug, Clone)] pub struct IngestServiceClient { - inner: Box, + inner: InnerIngestServiceClient, } +#[derive(Debug, Clone)] +struct InnerIngestServiceClient(std::sync::Arc); impl IngestServiceClient { pub fn new(instance: T) -> Self where @@ -231,7 +226,9 @@ impl IngestServiceClient { > (), "`MockIngestService` must be wrapped in a `MockIngestServiceWrapper`: use `IngestServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerIngestServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -290,7 +287,7 @@ impl IngestServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockIngestService) -> Self { let mock_wrapper = mock_ingest_service::MockIngestServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -301,39 +298,39 @@ impl IngestServiceClient { } #[async_trait::async_trait] impl IngestService for IngestServiceClient { - async fn ingest(&mut self, request: IngestRequest) -> crate::Result { - self.inner.ingest(request).await + async fn ingest(&self, request: IngestRequest) -> crate::Result { + self.inner.0.ingest(request).await } - async fn fetch(&mut self, request: FetchRequest) -> crate::Result { - self.inner.fetch(request).await + async fn fetch(&self, request: FetchRequest) -> crate::Result { + self.inner.0.fetch(request).await } - async fn tail(&mut self, request: TailRequest) -> crate::Result { - self.inner.tail(request).await + async fn tail(&self, request: TailRequest) -> crate::Result { + self.inner.0.tail(request).await } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_ingest_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockIngestServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl IngestService for MockIngestServiceWrapper { async fn ingest( - &mut self, + &self, request: super::IngestRequest, ) -> crate::Result { self.inner.lock().await.ingest(request).await } async fn fetch( - &mut self, + &self, request: super::FetchRequest, ) -> crate::Result { self.inner.lock().await.fetch(request).await } async fn tail( - &mut self, + &self, request: super::TailRequest, ) -> crate::Result { self.inner.lock().await.tail(request).await @@ -343,7 +340,7 @@ pub mod mock_ingest_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerIngestServiceClient { type Response = IngestResponse; type Error = crate::IngestServiceError; type Future = BoxFuture; @@ -354,12 +351,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: IngestRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.ingest(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.ingest(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngestServiceClient { type Response = FetchResponse; type Error = crate::IngestServiceError; type Future = BoxFuture; @@ -370,12 +367,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: FetchRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.fetch(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.fetch(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngestServiceClient { type Response = FetchResponse; type Error = crate::IngestServiceError; type Future = BoxFuture; @@ -386,15 +383,16 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: TailRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.tail(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.tail(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct IngestServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerIngestServiceClient, ingest_svc: quickwit_common::tower::BoxService< IngestRequest, IngestResponse, @@ -411,26 +409,16 @@ struct IngestServiceTowerServiceStack { crate::IngestServiceError, >, } -impl Clone for IngestServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - ingest_svc: self.ingest_svc.clone(), - fetch_svc: self.fetch_svc.clone(), - tail_svc: self.tail_svc.clone(), - } - } -} #[async_trait::async_trait] impl IngestService for IngestServiceTowerServiceStack { - async fn ingest(&mut self, request: IngestRequest) -> crate::Result { - self.ingest_svc.ready().await?.call(request).await + async fn ingest(&self, request: IngestRequest) -> crate::Result { + self.ingest_svc.clone().ready().await?.call(request).await } - async fn fetch(&mut self, request: FetchRequest) -> crate::Result { - self.fetch_svc.ready().await?.call(request).await + async fn fetch(&self, request: FetchRequest) -> crate::Result { + self.fetch_svc.clone().ready().await?.call(request).await } - async fn tail(&mut self, request: TailRequest) -> crate::Result { - self.tail_svc.ready().await?.call(request).await + async fn tail(&self, request: TailRequest) -> crate::Result { + self.tail_svc.clone().ready().await?.call(request).await } } type IngestLayer = quickwit_common::tower::BoxLayer< @@ -614,7 +602,8 @@ impl IngestServiceTowerLayerStack { where T: IngestService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerIngestServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -622,23 +611,21 @@ impl IngestServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> IngestServiceClient { - self.build_from_boxed( - Box::new(IngestServiceClient::from_channel(addr, channel, max_message_size)), - ) + let client = IngestServiceClient::from_channel(addr, channel, max_message_size); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> IngestServiceClient { - self.build_from_boxed( - Box::new( - IngestServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = IngestServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -648,22 +635,27 @@ impl IngestServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, IngestServiceMailbox: IngestService, { - self.build_from_boxed(Box::new(IngestServiceMailbox::new(mailbox))) + let inner_client = InnerIngestServiceClient( + std::sync::Arc::new(IngestServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: MockIngestService) -> IngestServiceClient { - self.build_from_boxed(Box::new(IngestServiceClient::from_mock(mock))) + let client = IngestServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerIngestServiceClient, ) -> IngestServiceClient { let ingest_svc = self .ingest_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let fetch_svc = self @@ -671,7 +663,7 @@ impl IngestServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tail_svc = self @@ -679,11 +671,11 @@ impl IngestServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = IngestServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, ingest_svc, fetch_svc, tail_svc, @@ -782,14 +774,14 @@ where Future = BoxFuture, >, { - async fn ingest(&mut self, request: IngestRequest) -> crate::Result { - self.call(request).await + async fn ingest(&self, request: IngestRequest) -> crate::Result { + self.clone().call(request).await } - async fn fetch(&mut self, request: FetchRequest) -> crate::Result { - self.call(request).await + async fn fetch(&self, request: FetchRequest) -> crate::Result { + self.clone().call(request).await } - async fn tail(&mut self, request: TailRequest) -> crate::Result { - self.call(request).await + async fn tail(&self, request: TailRequest) -> crate::Result { + self.clone().call(request).await } } #[derive(Debug, Clone)] @@ -826,8 +818,9 @@ where + Send, T::Future: Send, { - async fn ingest(&mut self, request: IngestRequest) -> crate::Result { + async fn ingest(&self, request: IngestRequest) -> crate::Result { self.inner + .clone() .ingest(request) .await .map(|response| response.into_inner()) @@ -836,8 +829,9 @@ where IngestRequest::rpc_name(), )) } - async fn fetch(&mut self, request: FetchRequest) -> crate::Result { + async fn fetch(&self, request: FetchRequest) -> crate::Result { self.inner + .clone() .fetch(request) .await .map(|response| response.into_inner()) @@ -846,8 +840,9 @@ where FetchRequest::rpc_name(), )) } - async fn tail(&mut self, request: TailRequest) -> crate::Result { + async fn tail(&self, request: TailRequest) -> crate::Result { self.inner + .clone() .tail(request) .await .map(|response| response.into_inner()) @@ -859,14 +854,16 @@ where } #[derive(Debug)] pub struct IngestServiceGrpcServerAdapter { - inner: Box, + inner: InnerIngestServiceClient, } impl IngestServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: IngestService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerIngestServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -876,7 +873,7 @@ impl ingest_service_grpc_server::IngestServiceGrpc for IngestServiceGrpcServerAd request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .ingest(request.into_inner()) .await .map(tonic::Response::new) @@ -887,7 +884,7 @@ impl ingest_service_grpc_server::IngestServiceGrpc for IngestServiceGrpcServerAd request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .fetch(request.into_inner()) .await .map(tonic::Response::new) @@ -898,7 +895,7 @@ impl ingest_service_grpc_server::IngestServiceGrpc for IngestServiceGrpcServerAd request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .tail(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs index 79dac00bb2a..6511a13ecb7 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/fetch.rs @@ -457,7 +457,7 @@ async fn fault_tolerant_fetch_stream( 'outer: for (ingester_idx, ingester_id) in ingester_ids.iter().enumerate() { let failover_ingester_id_opt = ingester_ids.get(ingester_idx + 1); - let Some(mut ingester) = ingester_pool.get(ingester_id) else { + let Some(ingester) = ingester_pool.get(ingester_id) else { if let Some(failover_ingester_id) = failover_ingester_id_opt { warn!( client_id=%client_id, diff --git a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs index a5ec1cb717e..a54c9fb0791 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/ingester.rs @@ -386,7 +386,7 @@ impl Ingester { .try_send(open_message) .expect("channel should be open and have capacity"); - let mut ingester = self.ingester_pool.get(&follower_id).ok_or_else(|| { + let ingester = self.ingester_pool.get(&follower_id).ok_or_else(|| { let message = format!("ingester `{follower_id}` is unavailable"); IngestV2Error::Unavailable(message) })?; @@ -422,7 +422,7 @@ impl Ingester { } async fn persist_inner( - &mut self, + &self, persist_request: PersistRequest, ) -> IngestV2Result { if persist_request.leader_id != self.self_node_id { @@ -794,7 +794,7 @@ impl Ingester { /// Opens a replication stream, which is a bi-directional gRPC stream. The client-side stream async fn open_replication_stream_inner( - &mut self, + &self, mut syn_replication_stream: quickwit_common::ServiceStream, ) -> IngestV2Result> { let open_replication_stream_request = syn_replication_stream @@ -845,7 +845,7 @@ impl Ingester { } async fn open_fetch_stream_inner( - &mut self, + &self, open_fetch_stream_request: OpenFetchStreamRequest, ) -> IngestV2Result>> { let queue_id = open_fetch_stream_request.queue_id(); @@ -876,7 +876,7 @@ impl Ingester { } async fn open_observation_stream_inner( - &mut self, + &self, _open_observation_stream_request: OpenObservationStreamRequest, ) -> IngestV2Result> { let status_stream = ServiceStream::from(self.state.status_rx.clone()); @@ -892,7 +892,7 @@ impl Ingester { } async fn init_shards_inner( - &mut self, + &self, init_shards_request: InitShardsRequest, ) -> IngestV2Result { let mut state_guard = @@ -939,7 +939,7 @@ impl Ingester { } async fn truncate_shards_inner( - &mut self, + &self, truncate_shards_request: TruncateShardsRequest, ) -> IngestV2Result { if truncate_shards_request.ingester_id != self.self_node_id { @@ -972,7 +972,7 @@ impl Ingester { } async fn close_shards_inner( - &mut self, + &self, close_shards_request: CloseShardsRequest, ) -> IngestV2Result { let mut state_guard = @@ -994,7 +994,7 @@ impl Ingester { } async fn decommission_inner( - &mut self, + &self, _decommission_request: DecommissionRequest, ) -> IngestV2Result { info!("decommissioning ingester"); @@ -1065,10 +1065,7 @@ impl Ingester { #[async_trait] impl IngesterService for Ingester { - async fn persist( - &mut self, - persist_request: PersistRequest, - ) -> IngestV2Result { + async fn persist(&self, persist_request: PersistRequest) -> IngestV2Result { // If the request is local, the amount of memory it occupies is already // accounted for in the router. let request_size_bytes = persist_request @@ -1086,7 +1083,7 @@ impl IngesterService for Ingester { } async fn open_replication_stream( - &mut self, + &self, syn_replication_stream: quickwit_common::ServiceStream, ) -> IngestV2Result> { self.open_replication_stream_inner(syn_replication_stream) @@ -1094,7 +1091,7 @@ impl IngesterService for Ingester { } async fn open_fetch_stream( - &mut self, + &self, open_fetch_stream_request: OpenFetchStreamRequest, ) -> IngestV2Result>> { self.open_fetch_stream_inner(open_fetch_stream_request) @@ -1102,7 +1099,7 @@ impl IngesterService for Ingester { } async fn open_observation_stream( - &mut self, + &self, open_observation_stream_request: OpenObservationStreamRequest, ) -> IngestV2Result> { self.open_observation_stream_inner(open_observation_stream_request) @@ -1110,14 +1107,14 @@ impl IngesterService for Ingester { } async fn init_shards( - &mut self, + &self, init_shards_request: InitShardsRequest, ) -> IngestV2Result { self.init_shards_inner(init_shards_request).await } async fn retain_shards( - &mut self, + &self, request: RetainShardsRequest, ) -> IngestV2Result { let retain_queue_ids: HashSet = request @@ -1150,21 +1147,21 @@ impl IngesterService for Ingester { } async fn truncate_shards( - &mut self, + &self, truncate_shards_request: TruncateShardsRequest, ) -> IngestV2Result { self.truncate_shards_inner(truncate_shards_request).await } async fn close_shards( - &mut self, + &self, close_shards_request: CloseShardsRequest, ) -> IngestV2Result { self.close_shards_inner(close_shards_request).await } async fn decommission( - &mut self, + &self, decommission_request: DecommissionRequest, ) -> IngestV2Result { self.decommission_inner(decommission_request).await @@ -1199,7 +1196,7 @@ impl EventSubscriber for WeakIngesterState { } pub async fn wait_for_ingester_status( - mut ingester: impl IngesterService, + ingester: impl IngesterService, status: IngesterStatus, ) -> anyhow::Result<()> { let mut observation_stream = ingester @@ -1218,7 +1215,7 @@ pub async fn wait_for_ingester_status( Ok(()) } -pub async fn wait_for_ingester_decommission(mut ingester: Ingester) -> anyhow::Result<()> { +pub async fn wait_for_ingester_decommission(ingester: Ingester) -> anyhow::Result<()> { let now = Instant::now(); ingester @@ -1578,7 +1575,7 @@ mod tests { #[tokio::test] async fn test_ingester_init_shards() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let shard = Shard { @@ -1620,7 +1617,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let index_uid2: IndexUid = IndexUid::for_test("test-index", 1); @@ -1731,7 +1728,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist_empty() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let persist_request = PersistRequest { @@ -1798,7 +1795,7 @@ mod tests { let scenario = fail::FailScenario::setup(); fail::cfg("ingester:append_records", "return").unwrap(); - let (_ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let mut state_guard = ingester.state.lock_fully().await.unwrap(); let index_uid: IndexUid = IndexUid::for_test("test-index", 0); @@ -1857,7 +1854,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist_deletes_dangling_shard() { - let (_ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let mut state_guard = ingester.state.lock_fully().await.unwrap(); let index_uid: IndexUid = IndexUid::for_test("test-index", 0); @@ -1911,7 +1908,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist_replicate() { - let (leader_ctx, mut leader) = IngesterForTest::default() + let (leader_ctx, leader) = IngesterForTest::default() .with_node_id("test-leader") .with_replication() .build() @@ -2070,7 +2067,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist_replicate_grpc() { - let (leader_ctx, mut leader) = IngesterForTest::default() + let (leader_ctx, leader) = IngesterForTest::default() .with_node_id("test-leader") .with_replication() .build() @@ -2254,7 +2251,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist_shard_closed() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let queue_id_01 = queue_id(&index_uid, "test-source", &ShardId::from(1)); let solo_shard = IngesterShard::new_solo( @@ -2305,7 +2302,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist_rate_limited() { - let (ingester_ctx, mut ingester) = IngesterForTest::default() + let (ingester_ctx, ingester) = IngesterForTest::default() .with_rate_limiter_settings(RateLimiterSettings { burst_limit: 0, rate_limit: ConstantRate::bytes_per_sec(ByteSize(0)), @@ -2377,7 +2374,7 @@ mod tests { #[tokio::test] async fn test_ingester_persist_resource_exhausted() { - let (ingester_ctx, mut ingester) = IngesterForTest::default() + let (ingester_ctx, ingester) = IngesterForTest::default() .with_disk_capacity(ByteSize(0)) .build() .await; @@ -2447,7 +2444,7 @@ mod tests { #[tokio::test] async fn test_ingester_open_replication_stream() { - let (_ingester_ctx, mut ingester) = IngesterForTest::default() + let (_ingester_ctx, ingester) = IngesterForTest::default() .with_node_id("test-follower") .build() .await; @@ -2481,7 +2478,7 @@ mod tests { #[tokio::test] async fn test_ingester_open_fetch_stream() { - let (_ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let open_fetch_stream_request = OpenFetchStreamRequest { @@ -2595,7 +2592,7 @@ mod tests { #[tokio::test] async fn test_ingester_truncate_shards() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let shard_01 = Shard { @@ -2709,7 +2706,7 @@ mod tests { #[tokio::test] async fn test_ingester_truncate_shards_deletes_dangling_shards() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let queue_id = queue_id(&index_uid, "test-source", &ShardId::from(1)); @@ -2859,7 +2856,7 @@ mod tests { #[tokio::test] async fn test_ingester_retain_shards() { - let (_ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let shard_17 = Shard { @@ -2930,7 +2927,7 @@ mod tests { #[tokio::test] async fn test_ingester_close_shards() { - let (_ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (_ingester_ctx, ingester) = IngesterForTest::default().build().await; let index_uid: IndexUid = IndexUid::for_test("test-index", 0); let shard = Shard { @@ -3014,7 +3011,7 @@ mod tests { #[tokio::test] async fn test_ingester_open_observation_stream() { - let (ingester_ctx, mut ingester) = IngesterForTest::default().build().await; + let (ingester_ctx, ingester) = IngesterForTest::default().build().await; let mut observation_stream = ingester .open_observation_stream(OpenObservationStreamRequest {}) diff --git a/quickwit/quickwit-ingest/src/ingest_v2/router.rs b/quickwit/quickwit-ingest/src/ingest_v2/router.rs index fde9419c1e3..fefdffb2834 100644 --- a/quickwit/quickwit-ingest/src/ingest_v2/router.rs +++ b/quickwit/quickwit-ingest/src/ingest_v2/router.rs @@ -206,7 +206,7 @@ impl IngestRouter { } async fn populate_routing_table_debounced( - &mut self, + &self, workbench: &mut IngestWorkbench, debounced_request: DebouncedGetOrCreateOpenShardsRequest, ) { @@ -221,7 +221,7 @@ impl IngestRouter { /// Issues a [`GetOrCreateOpenShardsRequest`] request to the control plane and populates the /// shard table according to the response received. async fn populate_routing_table( - &mut self, + &self, workbench: &mut IngestWorkbench, request: GetOrCreateOpenShardsRequest, ) { @@ -263,7 +263,7 @@ impl IngestRouter { } async fn process_persist_results( - &mut self, + &self, workbench: &mut IngestWorkbench, mut persist_futures: FuturesUnordered>, ) { @@ -332,7 +332,7 @@ impl IngestRouter { } } - async fn batch_persist(&mut self, workbench: &mut IngestWorkbench, commit_type: CommitTypeV2) { + async fn batch_persist(&self, workbench: &mut IngestWorkbench, commit_type: CommitTypeV2) { let debounced_request = self .make_get_or_create_open_shard_request(workbench, &self.ingester_pool) .await; @@ -381,7 +381,7 @@ impl IngestRouter { .iter() .map(|subrequest| subrequest.subrequest_id) .collect(); - let Some(mut ingester) = self.ingester_pool.get(&leader_id) else { + let Some(ingester) = self.ingester_pool.get(&leader_id) else { no_shards_available_subrequest_ids.extend(subrequest_ids); continue; }; @@ -421,7 +421,7 @@ impl IngestRouter { } async fn retry_batch_persist( - &mut self, + &self, ingest_request: IngestRequestV2, max_num_attempts: usize, ) -> IngestV2Result { @@ -435,7 +435,7 @@ impl IngestRouter { } async fn ingest_timeout( - &mut self, + &self, ingest_request: IngestRequestV2, timeout_duration: Duration, ) -> IngestV2Result { @@ -465,10 +465,7 @@ impl IngestRouter { #[async_trait] impl IngestRouterService for IngestRouter { - async fn ingest( - &mut self, - ingest_request: IngestRequestV2, - ) -> IngestV2Result { + async fn ingest(&self, ingest_request: IngestRequestV2) -> IngestV2Result { let request_size_bytes = ingest_request.num_bytes(); let mut gauge_guard = GaugeGuard::from_gauge(&MEMORY_METRICS.in_flight.ingest_router); @@ -820,7 +817,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(mock_control_plane); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -934,7 +931,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(mock_control_plane); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -992,7 +989,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(mock_control_plane); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -1021,7 +1018,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -1072,7 +1069,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -1123,7 +1120,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -1209,7 +1206,7 @@ mod tests { ingester_pool.insert("test-ingester-1".into(), IngesterServiceClient::mocked()); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -1288,7 +1285,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), @@ -1502,7 +1499,7 @@ mod tests { let control_plane = ControlPlaneServiceClient::from_mock(MockControlPlaneService::new()); let ingester_pool = IngesterPool::default(); let replication_factor = 1; - let mut router = IngestRouter::new( + let router = IngestRouter::new( self_node_id, control_plane, ingester_pool.clone(), diff --git a/quickwit/quickwit-integration-tests/src/tests/index_update_tests.rs b/quickwit/quickwit-integration-tests/src/tests/index_update_tests.rs index 5ddc8a034ec..d9bfdbac5eb 100644 --- a/quickwit/quickwit-integration-tests/src/tests/index_update_tests.rs +++ b/quickwit/quickwit-integration-tests/src/tests/index_update_tests.rs @@ -21,9 +21,8 @@ use std::collections::HashSet; use std::time::Duration; use quickwit_config::service::QuickwitService; -use quickwit_config::SearchSettings; use quickwit_rest_client::rest_client::CommitType; -use quickwit_serve::{IndexUpdates, SearchRequestQueryString}; +use quickwit_serve::SearchRequestQueryString; use serde_json::json; use crate::ingest_json; @@ -121,12 +120,21 @@ async fn test_update_on_multi_nodes_cluster() { .indexes() .update( "my-updatable-index", - IndexUpdates { - search_settings: SearchSettings { - default_search_fields: vec!["title".to_string(), "body".to_string()], - }, - retention_policy_opt: None, - }, + r#" + version: 0.8 + index_id: my-updatable-index + doc_mapping: + field_mappings: + - name: title + type: text + - name: body + type: text + indexing_settings: + commit_timeout_secs: 1 + search_settings: + default_search_fields: [title, body] + "#, + quickwit_config::ConfigFormat::Yaml, ) .await .unwrap(); diff --git a/quickwit/quickwit-jaeger/src/integration_tests.rs b/quickwit/quickwit-jaeger/src/integration_tests.rs index 54386cd3973..380de46db9c 100644 --- a/quickwit/quickwit-jaeger/src/integration_tests.rs +++ b/quickwit/quickwit-jaeger/src/integration_tests.rs @@ -388,7 +388,7 @@ async fn searcher_for_test( async fn setup_traces_index( temp_dir: &TempDir, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ingester_service: &Mailbox, indexer_service: &Mailbox, ) { diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs index 50fc78b7218..5c25c7ae1c7 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_pipeline.rs @@ -351,7 +351,7 @@ mod tests { serde_json::json!({"body": "delete", "ts": 0 }), ]; test_sandbox.add_documents(docs).await?; - let mut metastore = test_sandbox.metastore(); + let metastore = test_sandbox.metastore(); metastore .create_delete_task(DeleteQuery { index_uid: Some(index_uid.clone()), diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs index 4228a96c78e..659438c2b7d 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_planner.rs @@ -467,7 +467,7 @@ mod tests { for doc in docs { test_sandbox.add_documents(vec![doc]).await?; } - let mut metastore = test_sandbox.metastore(); + let metastore = test_sandbox.metastore(); let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_metadata = metastore .index_metadata(index_metadata_request) diff --git a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs index 0f9a1190393..cd39ff6769e 100644 --- a/quickwit/quickwit-janitor/src/actors/delete_task_service.rs +++ b/quickwit/quickwit-janitor/src/actors/delete_task_service.rs @@ -240,7 +240,7 @@ mod tests { "#; let test_sandbox = TestSandbox::create(index_id, doc_mapping_yaml, "{}", &["body"]).await?; let index_uid = test_sandbox.index_uid(); - let mut metastore = test_sandbox.metastore(); + let metastore = test_sandbox.metastore(); let mock_search_service = MockSearchService::new(); let searcher_pool = searcher_pool_for_test([("127.0.0.1:1000", mock_search_service)]); let search_job_placer = SearchJobPlacer::new(searcher_pool); diff --git a/quickwit/quickwit-janitor/src/retention_policy_execution.rs b/quickwit/quickwit-janitor/src/retention_policy_execution.rs index 10ea5963cd7..93efeb09975 100644 --- a/quickwit/quickwit-janitor/src/retention_policy_execution.rs +++ b/quickwit/quickwit-janitor/src/retention_policy_execution.rs @@ -43,7 +43,7 @@ use crate::actors::RetentionPolicyExecutor; /// * `ctx_opt` - A context for reporting progress (only useful within quickwit actor). pub async fn run_execute_retention_policy( index_uid: IndexUid, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, retention_policy: &RetentionPolicy, ctx: &ActorContext, ) -> anyhow::Result> { diff --git a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs index 7f16211c462..c7183acfbd0 100644 --- a/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/control_plane_metastore.rs @@ -72,14 +72,14 @@ impl MetastoreService for ControlPlaneMetastore { self.metastore.endpoints() } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { self.metastore.check_connectivity().await } // Proxied metastore API calls. async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> MetastoreResult { let response = self.control_plane.create_index(request).await?; @@ -92,38 +92,29 @@ impl MetastoreService for ControlPlaneMetastore { // doc mapping or merge policy of an index, so we've already set up the proxy here since calling // `update_index` is very infrequent anyway. async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> MetastoreResult { let response = self.control_plane.update_index(request).await?; Ok(response) } - async fn delete_index( - &mut self, - request: DeleteIndexRequest, - ) -> MetastoreResult { + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { let response = self.control_plane.delete_index(request).await?; Ok(response) } - async fn add_source(&mut self, request: AddSourceRequest) -> MetastoreResult { + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { let response = self.control_plane.add_source(request).await?; Ok(response) } - async fn toggle_source( - &mut self, - request: ToggleSourceRequest, - ) -> MetastoreResult { + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let response = self.control_plane.clone().toggle_source(request).await?; Ok(response) } - async fn delete_source( - &mut self, - request: DeleteSourceRequest, - ) -> MetastoreResult { + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { let response = self.control_plane.delete_source(request).await?; Ok(response) } @@ -131,70 +122,64 @@ impl MetastoreService for ControlPlaneMetastore { // Other metastore API calls. async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> MetastoreResult { self.metastore.index_metadata(request).await } async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> MetastoreResult { self.metastore.indexes_metadata(request).await } async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> MetastoreResult { self.metastore.list_indexes_metadata(request).await } - async fn stage_splits( - &mut self, - request: StageSplitsRequest, - ) -> MetastoreResult { + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { self.metastore.stage_splits(request).await } async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> MetastoreResult { self.metastore.publish_splits(request).await } async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> MetastoreResult> { self.metastore.list_splits(request).await } async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> MetastoreResult { self.metastore.list_stale_splits(request).await } async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> MetastoreResult { self.metastore.mark_splits_for_deletion(request).await } - async fn delete_splits( - &mut self, - request: DeleteSplitsRequest, - ) -> MetastoreResult { + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { self.metastore.delete_splits(request).await } async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> MetastoreResult { self.metastore.reset_source_checkpoint(request).await @@ -202,29 +187,26 @@ impl MetastoreService for ControlPlaneMetastore { // Delete tasks API - async fn create_delete_task( - &mut self, - delete_query: DeleteQuery, - ) -> MetastoreResult { + async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { self.metastore.create_delete_task(delete_query).await } async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> MetastoreResult { self.metastore.last_delete_opstamp(request).await } async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> MetastoreResult { self.metastore.update_splits_delete_opstamp(request).await } async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> MetastoreResult { self.metastore.list_delete_tasks(request).await @@ -232,29 +214,23 @@ impl MetastoreService for ControlPlaneMetastore { // Shard API - async fn open_shards( - &mut self, - request: OpenShardsRequest, - ) -> MetastoreResult { + async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { self.metastore.open_shards(request).await } async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> MetastoreResult { self.metastore.acquire_shards(request).await } - async fn list_shards( - &mut self, - request: ListShardsRequest, - ) -> MetastoreResult { + async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { self.metastore.list_shards(request).await } async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> MetastoreResult { self.metastore.delete_shards(request).await @@ -263,35 +239,35 @@ impl MetastoreService for ControlPlaneMetastore { // Index Template API async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> MetastoreResult { self.metastore.create_index_template(request).await } async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> MetastoreResult { self.metastore.get_index_template(request).await } async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> MetastoreResult { self.metastore.find_index_template_matches(request).await } async fn list_index_templates( - &mut self, + &self, request: ListIndexTemplatesRequest, ) -> MetastoreResult { self.metastore.list_index_templates(request).await } async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> MetastoreResult { self.metastore.delete_index_templates(request).await diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs index 2a480e3d716..68a6dc56a77 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/file_backed_index/mod.rs @@ -30,7 +30,9 @@ use std::ops::Bound; use itertools::Itertools; use quickwit_common::pretty::PrettySample; -use quickwit_config::{RetentionPolicy, SearchSettings, SourceConfig, INGEST_V2_SOURCE_ID}; +use quickwit_config::{ + IndexingSettings, RetentionPolicy, SearchSettings, SourceConfig, INGEST_V2_SOURCE_ID, +}; use quickwit_proto::metastore::{ AcquireShardsRequest, AcquireShardsResponse, DeleteQuery, DeleteShardsRequest, DeleteShardsResponse, DeleteTask, EntityKind, ListShardsSubrequest, ListShardsSubresponse, @@ -223,6 +225,11 @@ impl FileBackedIndex { self.metadata.set_search_settings(search_settings) } + /// Replaces the indexing settings in the index config, returning whether a mutation occurred. + pub fn set_indexing_settings(&mut self, search_settings: IndexingSettings) -> bool { + self.metadata.set_indexing_settings(search_settings) + } + /// Stages a single split. /// /// If a split already exists and is in the [SplitState::Staged] state, diff --git a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs index 244033f8177..6157f20fed7 100644 --- a/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/file_backed/mod.rs @@ -340,7 +340,7 @@ impl FileBackedMetastore { } async fn index_metadata_inner( - &mut self, + &self, index_id_opt: Option, index_uid_opt: Option, ) -> Result, Option)> { @@ -407,7 +407,7 @@ impl FileBackedMetastore { #[async_trait] impl MetastoreService for FileBackedMetastore { - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { self.storage.exists(Path::new(MANIFEST_FILE_NAME)).await?; Ok(()) } @@ -419,7 +419,7 @@ impl MetastoreService for FileBackedMetastore { /// ------------------------------------------------------------------------------- /// Mutations over the high-level index. async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> MetastoreResult { let index_config = request.deserialize_index_config()?; @@ -499,17 +499,19 @@ impl MetastoreService for FileBackedMetastore { } async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> MetastoreResult { let retention_policy_opt = request.deserialize_retention_policy()?; let search_settings = request.deserialize_search_settings()?; + let indexing_settings = request.deserialize_indexing_settings()?; let index_uid = request.index_uid(); let index_metadata = self .mutate(index_uid, |index| { let mut mutation_occurred = index.set_retention_policy(retention_policy_opt); mutation_occurred |= index.set_search_settings(search_settings); + mutation_occurred |= index.set_indexing_settings(indexing_settings); let index_metadata = index.metadata().clone(); @@ -523,10 +525,7 @@ impl MetastoreService for FileBackedMetastore { IndexMetadataResponse::try_from_index_metadata(&index_metadata) } - async fn delete_index( - &mut self, - request: DeleteIndexRequest, - ) -> MetastoreResult { + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { // We pick the outer lock here, so that we enter a critical section. let mut state_wlock_guard = self.state.write().await; @@ -580,10 +579,7 @@ impl MetastoreService for FileBackedMetastore { /// ------------------------------------------------------------------------------- /// Mutations over a single index - async fn stage_splits( - &mut self, - request: StageSplitsRequest, - ) -> MetastoreResult { + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { let splits_metadata = request.deserialize_splits_metadata()?; let index_uid = request.index_uid(); @@ -617,7 +613,7 @@ impl MetastoreService for FileBackedMetastore { } async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> MetastoreResult { let index_checkpoint_delta: Option = @@ -637,7 +633,7 @@ impl MetastoreService for FileBackedMetastore { } async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> MetastoreResult { let index_uid = request.index_uid().clone(); @@ -659,10 +655,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } - async fn delete_splits( - &mut self, - request: DeleteSplitsRequest, - ) -> MetastoreResult { + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { let index_uid = request.index_uid().clone(); self.mutate(&index_uid, |index| { @@ -673,7 +666,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } - async fn add_source(&mut self, request: AddSourceRequest) -> MetastoreResult { + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { let source_config = request.deserialize_source_config()?; let index_uid = request.index_uid(); @@ -685,10 +678,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } - async fn toggle_source( - &mut self, - request: ToggleSourceRequest, - ) -> MetastoreResult { + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let index_uid = request.index_uid(); self.mutate(index_uid, |index| { @@ -700,10 +690,7 @@ impl MetastoreService for FileBackedMetastore { Ok(EmptyResponse {}) } - async fn delete_source( - &mut self, - request: DeleteSourceRequest, - ) -> MetastoreResult { + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { let index_uid = request.index_uid(); self.mutate(index_uid, |index| { @@ -715,7 +702,7 @@ impl MetastoreService for FileBackedMetastore { } async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> MetastoreResult { let index_uid = request.index_uid(); @@ -735,7 +722,7 @@ impl MetastoreService for FileBackedMetastore { /// Streams of splits for the given request. /// No error is returned if any of the requested `index_uid` does not exist. async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> MetastoreResult> { let splits = self.list_splits_inner(request).await?; @@ -748,7 +735,7 @@ impl MetastoreService for FileBackedMetastore { } async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> MetastoreResult { let list_splits_query = ListSplitsQuery::for_index(request.index_uid().clone()) @@ -764,7 +751,7 @@ impl MetastoreService for FileBackedMetastore { } async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> MetastoreResult { let index_metadata = self @@ -776,7 +763,7 @@ impl MetastoreService for FileBackedMetastore { } async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> MetastoreResult { let mut indexes_metadata: Vec = @@ -786,7 +773,7 @@ impl MetastoreService for FileBackedMetastore { let mut index_metadata_futures = FuturesUnordered::new(); for subrequest in request.subrequests { - let mut metastore = self.clone(); + let metastore = self.clone(); let index_metadata_future = async move { metastore .index_metadata_inner(subrequest.index_id, subrequest.index_uid) @@ -822,7 +809,7 @@ impl MetastoreService for FileBackedMetastore { } async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> MetastoreResult { // Done in two steps: @@ -861,10 +848,7 @@ impl MetastoreService for FileBackedMetastore { // Shard API - async fn open_shards( - &mut self, - request: OpenShardsRequest, - ) -> MetastoreResult { + async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { let mut response = OpenShardsResponse { subresponses: Vec::with_capacity(request.subrequests.len()), }; @@ -885,7 +869,7 @@ impl MetastoreService for FileBackedMetastore { } async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> MetastoreResult { let index_uid = request.index_uid().clone(); @@ -896,7 +880,7 @@ impl MetastoreService for FileBackedMetastore { } async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> MetastoreResult { let index_uid = request.index_uid().clone(); @@ -906,10 +890,7 @@ impl MetastoreService for FileBackedMetastore { Ok(response) } - async fn list_shards( - &mut self, - request: ListShardsRequest, - ) -> MetastoreResult { + async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { let mut subresponses = Vec::with_capacity(request.subrequests.len()); for subrequest in request.subrequests { @@ -927,7 +908,7 @@ impl MetastoreService for FileBackedMetastore { /// Delete tasks async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> MetastoreResult { let last_delete_opstamp = self @@ -936,10 +917,7 @@ impl MetastoreService for FileBackedMetastore { Ok(LastDeleteOpstampResponse::new(last_delete_opstamp)) } - async fn create_delete_task( - &mut self, - delete_query: DeleteQuery, - ) -> MetastoreResult { + async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { let index_uid = delete_query.index_uid().clone(); let delete_task = self .mutate(&index_uid, |index| { @@ -952,7 +930,7 @@ impl MetastoreService for FileBackedMetastore { } async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> MetastoreResult { let index_uid = request.index_uid(); @@ -972,7 +950,7 @@ impl MetastoreService for FileBackedMetastore { } async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> MetastoreResult { let index_uid = request.index_uid(); @@ -989,7 +967,7 @@ impl MetastoreService for FileBackedMetastore { // Index Template API async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> MetastoreResult { let index_template: IndexTemplate = @@ -1046,7 +1024,7 @@ impl MetastoreService for FileBackedMetastore { } async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> MetastoreResult { let inner_rlock_guard = self.state.read().await; @@ -1066,7 +1044,7 @@ impl MetastoreService for FileBackedMetastore { } async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> MetastoreResult { let inner_rlock_guard = self.state.read().await; @@ -1097,7 +1075,7 @@ impl MetastoreService for FileBackedMetastore { } async fn list_index_templates( - &mut self, + &self, _request: ListIndexTemplatesRequest, ) -> MetastoreResult { let inner_rlock_guard = self.state.read().await; @@ -1114,7 +1092,7 @@ impl MetastoreService for FileBackedMetastore { } async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> MetastoreResult { let mut evicted_templates = Vec::with_capacity(request.template_ids.len()); @@ -1170,7 +1148,7 @@ async fn get_index_mutex( } async fn get_index_metadata( - mut metastore: FileBackedMetastore, + metastore: FileBackedMetastore, index_id: IndexId, ) -> MetastoreResult> { let request = IndexMetadataRequest::for_index_id(index_id); @@ -1235,7 +1213,7 @@ mod tests { #[async_trait] impl ReadWriteShardsForTest for FileBackedMetastore { async fn insert_shards( - &mut self, + &self, index_uid: &IndexUid, source_id: &SourceId, shards: Vec, @@ -1262,7 +1240,7 @@ mod tests { #[tokio::test] async fn test_metastore_connectivity_and_endpoints() { - let mut metastore = FileBackedMetastore::default_for_test().await; + let metastore = FileBackedMetastore::default_for_test().await; metastore.check_connectivity().await.unwrap(); assert_eq!(metastore.endpoints()[0].protocol(), Protocol::Ram); } @@ -1283,7 +1261,7 @@ mod tests { assert!(path == Path::new("manifest.json")); block_on(ram_storage_clone.put(path, put_payload)) }); - let mut metastore = FileBackedMetastore::try_new(Arc::new(mock_storage), None) + let metastore = FileBackedMetastore::try_new(Arc::new(mock_storage), None) .await .unwrap(); @@ -1306,7 +1284,7 @@ mod tests { #[tokio::test] async fn test_file_backed_metastore_get_index() { - let mut metastore = FileBackedMetastore::default_for_test().await; + let metastore = FileBackedMetastore::default_for_test().await; // Create index let index_id = "test-index"; @@ -1383,7 +1361,7 @@ mod tests { Err(StorageErrorKind::Io .with_error(anyhow::anyhow!("Oops. Some network problem maybe?"))) }); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_config = IndexConfig::for_test("test-index", "ram:///indexes/test-index"); let create_index_request = @@ -1481,7 +1459,7 @@ mod tests { #[tokio::test] async fn test_file_backed_metastore_write_directly_visible() -> MetastoreResult<()> { - let mut metastore = FileBackedMetastore::default_for_test().await; + let metastore = FileBackedMetastore::default_for_test().await; let index_config = IndexConfig::for_test("test-index", "ram:///indexes/test-index"); let create_index_request = @@ -1526,11 +1504,11 @@ mod tests { async fn test_file_backed_metastore_polling() -> MetastoreResult<()> { let storage = Arc::new(RamStorage::default()); - let mut metastore_write = FileBackedMetastore::try_new(storage.clone(), None) + let metastore_write = FileBackedMetastore::try_new(storage.clone(), None) .await .unwrap(); let polling_interval = Duration::from_millis(20); - let mut metastore_read = FileBackedMetastore::try_new(storage, Some(polling_interval)) + let metastore_read = FileBackedMetastore::try_new(storage, Some(polling_interval)) .await .unwrap(); @@ -1602,7 +1580,7 @@ mod tests { #[tokio::test(flavor = "multi_thread", worker_threads = 3)] async fn test_file_backed_metastore_race_condition() { - let mut metastore = FileBackedMetastore::default_for_test().await; + let metastore = FileBackedMetastore::default_for_test().await; let index_config = IndexConfig::for_test("test-index", "ram:///indexes/test-index"); let create_index_request = @@ -1615,7 +1593,7 @@ mod tests { let mut random_generator = rand::thread_rng(); for i in 1..=20 { let sleep_duration = Duration::from_millis(random_generator.gen_range(0..=200)); - let mut metastore = metastore.clone(); + let metastore = metastore.clone(); let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); let handle = tokio::spawn({ let index_uid = index_uid.clone(); @@ -1675,7 +1653,7 @@ mod tests { #[tokio::test(flavor = "multi_thread", worker_threads = 3)] async fn test_file_backed_metastore_list_indexes_race_condition() { - let mut metastore = FileBackedMetastore::default_for_test().await; + let metastore = FileBackedMetastore::default_for_test().await; let mut index_uids = Vec::new(); for idx in 0..10 { let index_uid = IndexUid::new_with_random_ulid(&format!("test-index-{idx}")); @@ -1698,7 +1676,7 @@ mod tests { index_uid: Some(index_uid.clone()), }; { - let mut metastore = metastore.clone(); + let metastore = metastore.clone(); let handle = tokio::spawn(async move { metastore .list_indexes_metadata(ListIndexesMetadataRequest::all()) @@ -1708,7 +1686,7 @@ mod tests { handles.push(handle); } { - let mut metastore = metastore.clone(); + let metastore = metastore.clone(); let handle = tokio::spawn(async move { metastore.delete_index(delete_request).await.unwrap(); }); @@ -1747,7 +1725,7 @@ mod tests { .times(1) .returning(move |path| block_on(ram_storage.get_all(path))); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); // Create index. @@ -1798,7 +1776,7 @@ mod tests { .expect_get_all() .times(1) .returning(move |path| block_on(ram_storage.get_all(path))); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); // Create index @@ -1874,7 +1852,7 @@ mod tests { } return block_on(ram_storage_clone.put(path, put_payload)); }); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_config = IndexConfig::for_test(index_id, "ram:///indexes/test-index"); // Create index @@ -1927,7 +1905,7 @@ mod tests { .returning(move |path, put_payload| { return block_on(ram_storage_clone.put(path, put_payload)); }); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); // Delete index let delete_request = DeleteIndexRequest { @@ -1983,7 +1961,7 @@ mod tests { } return block_on(ram_storage_clone.put(path, put_payload)); }); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); // Delete index let delete_request = DeleteIndexRequest { @@ -2043,7 +2021,7 @@ mod tests { .await?; // Fetch alive indexes metadatas. - let mut metastore = FileBackedMetastore::try_new(ram_storage.clone(), None) + let metastore = FileBackedMetastore::try_new(ram_storage.clone(), None) .await .unwrap(); let indexes_metadata = metastore @@ -2097,7 +2075,7 @@ mod tests { #[tokio::test] async fn test_monotically_increasing_stamps_by_index() { let storage = RamStorage::default(); - let mut metastore = FileBackedMetastore::try_new(Arc::new(storage.clone()), None) + let metastore = FileBackedMetastore::try_new(Arc::new(storage.clone()), None) .await .unwrap(); let index_id = "test-index-increasing-stamps-by-index"; @@ -2129,7 +2107,7 @@ mod tests { assert_eq!(delete_task_2.opstamp, 2); // Create metastore with data already in the storage. - let mut new_metastore = FileBackedMetastore::try_new(Arc::new(storage), None) + let new_metastore = FileBackedMetastore::try_new(Arc::new(storage), None) .await .unwrap(); let delete_task_3 = new_metastore @@ -2184,7 +2162,7 @@ mod tests { Err(io_error) }); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_template = IndexTemplate::for_test("test-template", &["test-index-foo*"], 100); let index_template_json = serde_json::to_string(&index_template).unwrap(); @@ -2243,7 +2221,7 @@ mod tests { Err(io_error) }); - let mut metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); + let metastore = FileBackedMetastore::for_test(Arc::new(mock_storage)); let index_template = IndexTemplate::for_test("test-template", &["test-index-foo*"], 100); let index_template_json = serde_json::to_string(&index_template).unwrap(); diff --git a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs index f3e59fe90b6..4176be081d3 100644 --- a/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/index_metadata/mod.rs @@ -23,7 +23,9 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; use quickwit_common::uri::Uri; -use quickwit_config::{IndexConfig, RetentionPolicy, SearchSettings, SourceConfig}; +use quickwit_config::{ + IndexConfig, IndexingSettings, RetentionPolicy, SearchSettings, SourceConfig, +}; use quickwit_proto::metastore::{EntityKind, MetastoreError, MetastoreResult}; use quickwit_proto::types::{IndexUid, SourceId}; use serde::{Deserialize, Serialize}; @@ -108,7 +110,7 @@ impl IndexMetadata { } } - /// Replaces or removes the current search settings, returning whether a mutation occurred. + /// Replaces the current search settings, returning whether a mutation occurred. pub fn set_search_settings(&mut self, search_settings: SearchSettings) -> bool { if self.index_config.search_settings != search_settings { self.index_config.search_settings = search_settings; @@ -118,6 +120,16 @@ impl IndexMetadata { } } + /// Replaces the current indexing settings, returning whether a mutation occurred. + pub fn set_indexing_settings(&mut self, indexing_settings: IndexingSettings) -> bool { + if self.index_config.indexing_settings != indexing_settings { + self.index_config.indexing_settings = indexing_settings; + true + } else { + false + } + } + /// Adds a source to the index. Returns an error if the source already exists. pub fn add_source(&mut self, source_config: SourceConfig) -> MetastoreResult<()> { match self.sources.entry(source_config.source_id.clone()) { diff --git a/quickwit/quickwit-metastore/src/metastore/mod.rs b/quickwit/quickwit-metastore/src/metastore/mod.rs index 3ca762f6068..7378faa94fc 100644 --- a/quickwit/quickwit-metastore/src/metastore/mod.rs +++ b/quickwit/quickwit-metastore/src/metastore/mod.rs @@ -32,7 +32,9 @@ use futures::TryStreamExt; pub use index_metadata::IndexMetadata; use itertools::Itertools; use quickwit_common::thread_pool::run_cpu_intensive; -use quickwit_config::{IndexConfig, RetentionPolicy, SearchSettings, SourceConfig}; +use quickwit_config::{ + IndexConfig, IndexingSettings, RetentionPolicy, SearchSettings, SourceConfig, +}; use quickwit_doc_mapper::tag_pruning::TagFilterAst; use quickwit_proto::metastore::{ serde_utils, AddSourceRequest, CreateIndexRequest, CreateIndexResponse, DeleteTask, @@ -188,6 +190,7 @@ pub trait UpdateIndexRequestExt { index_uid: impl Into, search_settings: &SearchSettings, retention_policy_opt: &Option, + indexing_settings: &IndexingSettings, ) -> MetastoreResult; /// Deserializes the `search_settings_json` field of an [`UpdateIndexRequest`] into a @@ -197,6 +200,10 @@ pub trait UpdateIndexRequestExt { /// Deserializes the `retention_policy_json` field of an [`UpdateIndexRequest`] into a /// [`RetentionPolicy`] object. fn deserialize_retention_policy(&self) -> MetastoreResult>; + + /// Deserializes the `indexing_settings_json` field of an [`UpdateIndexRequest`] into a + /// [`IndexingSettings`] object. + fn deserialize_indexing_settings(&self) -> MetastoreResult; } impl UpdateIndexRequestExt for UpdateIndexRequest { @@ -204,17 +211,20 @@ impl UpdateIndexRequestExt for UpdateIndexRequest { index_uid: impl Into, search_settings: &SearchSettings, retention_policy_opt: &Option, + indexing_settings: &IndexingSettings, ) -> MetastoreResult { - let search_settings_json = serde_utils::to_json_str(&search_settings)?; + let search_settings_json = serde_utils::to_json_str(search_settings)?; let retention_policy_json = retention_policy_opt .as_ref() .map(serde_utils::to_json_str) .transpose()?; + let indexing_settings_json = serde_utils::to_json_str(indexing_settings)?; let update_request = UpdateIndexRequest { index_uid: Some(index_uid.into()), search_settings_json, retention_policy_json, + indexing_settings_json, }; Ok(update_request) } @@ -229,6 +239,10 @@ impl UpdateIndexRequestExt for UpdateIndexRequest { .map(|policy| serde_utils::from_json_str(policy)) .transpose() } + + fn deserialize_indexing_settings(&self) -> MetastoreResult { + serde_utils::from_json_str(&self.indexing_settings_json) + } } /// Helper trait to build a [`IndexMetadataResponse`] and deserialize its payload. diff --git a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs index c6fcfb45ada..9fbe443229b 100644 --- a/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs +++ b/quickwit/quickwit-metastore/src/metastore/postgres/metastore.rs @@ -344,7 +344,7 @@ where #[async_trait] impl MetastoreService for PostgresqlMetastore { - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { self.connection_pool.acquire().await?; Ok(()) } @@ -362,7 +362,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> MetastoreResult { let index_config = request.deserialize_index_config()?; @@ -393,17 +393,19 @@ impl MetastoreService for PostgresqlMetastore { } async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> MetastoreResult { let retention_policy_opt = request.deserialize_retention_policy()?; let search_settings = request.deserialize_search_settings()?; + let indexing_settings = request.deserialize_indexing_settings()?; let index_uid: IndexUid = request.index_uid().clone(); let updated_index_metadata = run_with_tx!(self.connection_pool, tx, { mutate_index_metadata::(tx, index_uid, |index_metadata| { let mut mutation_occurred = index_metadata.set_retention_policy(retention_policy_opt); mutation_occurred |= index_metadata.set_search_settings(search_settings); + mutation_occurred |= index_metadata.set_indexing_settings(indexing_settings); Ok(MutationOccurred::from(mutation_occurred)) }) .await @@ -413,7 +415,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> MetastoreResult { let pg_index_opt = if let Some(index_uid) = &request.index_uid { @@ -440,7 +442,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> MetastoreResult { const INDEXES_METADATA_QUERY: &str = include_str!("queries/indexes_metadata.sql"); @@ -514,7 +516,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> MetastoreResult { let sql = @@ -537,10 +539,7 @@ impl MetastoreService for PostgresqlMetastore { } #[instrument(skip_all, fields(index_id=%request.index_uid()))] - async fn delete_index( - &mut self, - request: DeleteIndexRequest, - ) -> MetastoreResult { + async fn delete_index(&self, request: DeleteIndexRequest) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); let delete_result = sqlx::query("DELETE FROM indexes WHERE index_uid = $1") .bind(&index_uid) @@ -557,10 +556,7 @@ impl MetastoreService for PostgresqlMetastore { } #[instrument(skip_all, fields(split_ids))] - async fn stage_splits( - &mut self, - request: StageSplitsRequest, - ) -> MetastoreResult { + async fn stage_splits(&self, request: StageSplitsRequest) -> MetastoreResult { let splits_metadata = request.deserialize_splits_metadata()?; if splits_metadata.is_empty() { @@ -667,7 +663,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> MetastoreResult { let checkpoint_delta_opt: Option = @@ -835,7 +831,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> MetastoreResult> { let list_splits_query = request.deserialize_list_splits_query()?; @@ -885,7 +881,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); @@ -960,10 +956,7 @@ impl MetastoreService for PostgresqlMetastore { } #[instrument(skip(self))] - async fn delete_splits( - &mut self, - request: DeleteSplitsRequest, - ) -> MetastoreResult { + async fn delete_splits(&self, request: DeleteSplitsRequest) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); let split_ids = request.split_ids; const DELETE_SPLITS_QUERY: &str = r#" @@ -1049,7 +1042,7 @@ impl MetastoreService for PostgresqlMetastore { } #[instrument(skip(self))] - async fn add_source(&mut self, request: AddSourceRequest) -> MetastoreResult { + async fn add_source(&self, request: AddSourceRequest) -> MetastoreResult { let source_config = request.deserialize_source_config()?; let index_uid: IndexUid = request.index_uid().clone(); run_with_tx!(self.connection_pool, tx, { @@ -1064,10 +1057,7 @@ impl MetastoreService for PostgresqlMetastore { } #[instrument(skip(self))] - async fn toggle_source( - &mut self, - request: ToggleSourceRequest, - ) -> MetastoreResult { + async fn toggle_source(&self, request: ToggleSourceRequest) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); run_with_tx!(self.connection_pool, tx, { mutate_index_metadata(tx, index_uid, |index_metadata| { @@ -1084,10 +1074,7 @@ impl MetastoreService for PostgresqlMetastore { } #[instrument(skip(self))] - async fn delete_source( - &mut self, - request: DeleteSourceRequest, - ) -> MetastoreResult { + async fn delete_source(&self, request: DeleteSourceRequest) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); let source_id = request.source_id.clone(); run_with_tx!(self.connection_pool, tx, { @@ -1115,7 +1102,7 @@ impl MetastoreService for PostgresqlMetastore { #[instrument(skip(self))] async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); @@ -1136,7 +1123,7 @@ impl MetastoreService for PostgresqlMetastore { /// Retrieves the last delete opstamp for a given `index_id`. #[instrument(skip(self))] async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> MetastoreResult { let max_opstamp: i64 = sqlx::query_scalar( @@ -1158,10 +1145,7 @@ impl MetastoreService for PostgresqlMetastore { /// Creates a delete task from a delete query. #[instrument(skip(self))] - async fn create_delete_task( - &mut self, - delete_query: DeleteQuery, - ) -> MetastoreResult { + async fn create_delete_task(&self, delete_query: DeleteQuery) -> MetastoreResult { let delete_query_json = serde_json::to_string(&delete_query).map_err(|error| { MetastoreError::JsonSerializeError { struct_name: "DeleteQuery".to_string(), @@ -1191,7 +1175,7 @@ impl MetastoreService for PostgresqlMetastore { /// Update splits delete opstamps. #[instrument(skip(self))] async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); @@ -1236,7 +1220,7 @@ impl MetastoreService for PostgresqlMetastore { /// Lists the delete tasks with opstamp > `opstamp_start`. #[instrument(skip(self))] async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); @@ -1264,7 +1248,7 @@ impl MetastoreService for PostgresqlMetastore { /// values. #[instrument(skip(self))] async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> MetastoreResult { let index_uid: IndexUid = request.index_uid().clone(); @@ -1297,10 +1281,7 @@ impl MetastoreService for PostgresqlMetastore { } // TODO: Issue a single SQL query. - async fn open_shards( - &mut self, - request: OpenShardsRequest, - ) -> MetastoreResult { + async fn open_shards(&self, request: OpenShardsRequest) -> MetastoreResult { let mut subresponses = Vec::with_capacity(request.subrequests.len()); for subrequest in request.subrequests { @@ -1315,7 +1296,7 @@ impl MetastoreService for PostgresqlMetastore { } async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> MetastoreResult { const ACQUIRE_SHARDS_QUERY: &str = include_str!("queries/shards/acquire.sql"); @@ -1338,10 +1319,7 @@ impl MetastoreService for PostgresqlMetastore { Ok(response) } - async fn list_shards( - &mut self, - request: ListShardsRequest, - ) -> MetastoreResult { + async fn list_shards(&self, request: ListShardsRequest) -> MetastoreResult { if request.subrequests.is_empty() { return Ok(Default::default()); } @@ -1414,7 +1392,7 @@ impl MetastoreService for PostgresqlMetastore { } async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> MetastoreResult { const DELETE_SHARDS_QUERY: &str = include_str!("queries/shards/delete.sql"); @@ -1488,7 +1466,7 @@ impl MetastoreService for PostgresqlMetastore { // Index Template API async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> MetastoreResult { const INSERT_INDEX_TEMPLATE_QUERY: &str = @@ -1548,7 +1526,7 @@ impl MetastoreService for PostgresqlMetastore { } async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> MetastoreResult { let pg_index_template_json: PgIndexTemplate = @@ -1568,7 +1546,7 @@ impl MetastoreService for PostgresqlMetastore { } async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> MetastoreResult { if request.index_ids.is_empty() { @@ -1598,7 +1576,7 @@ impl MetastoreService for PostgresqlMetastore { } async fn list_index_templates( - &mut self, + &self, _request: ListIndexTemplatesRequest, ) -> MetastoreResult { let pg_index_templates_json: Vec<(String,)> = sqlx::query_as( @@ -1617,7 +1595,7 @@ impl MetastoreService for PostgresqlMetastore { } async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> MetastoreResult { sqlx::query("DELETE FROM index_templates WHERE template_id = ANY($1)") @@ -1788,7 +1766,7 @@ mod tests { #[async_trait] impl ReadWriteShardsForTest for PostgresqlMetastore { async fn insert_shards( - &mut self, + &self, index_uid: &IndexUid, source_id: &SourceId, shards: Vec, @@ -1838,7 +1816,7 @@ mod tests { #[tokio::test] async fn test_metastore_connectivity_and_endpoints() { - let mut metastore = PostgresqlMetastore::default_for_test().await; + let metastore = PostgresqlMetastore::default_for_test().await; metastore.check_connectivity().await.unwrap(); assert_eq!(metastore.endpoints()[0].protocol(), Protocol::PostgreSQL); } diff --git a/quickwit/quickwit-metastore/src/tests/delete_task.rs b/quickwit/quickwit-metastore/src/tests/delete_task.rs index 1b4091a3dfb..5edc7fa082f 100644 --- a/quickwit/quickwit-metastore/src/tests/delete_task.rs +++ b/quickwit/quickwit-metastore/src/tests/delete_task.rs @@ -178,7 +178,7 @@ pub async fn test_metastore_last_delete_opstamp< pub async fn test_metastore_delete_index_with_tasks< MetastoreToTest: MetastoreServiceExt + DefaultForTest, >() { - let mut metastore = MetastoreToTest::default_for_test().await; + let metastore = MetastoreToTest::default_for_test().await; let index_id = append_random_suffix("delete-delete-tasks"); let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); diff --git a/quickwit/quickwit-metastore/src/tests/index.rs b/quickwit/quickwit-metastore/src/tests/index.rs index 48dbcd14655..613c7cd575d 100644 --- a/quickwit/quickwit-metastore/src/tests/index.rs +++ b/quickwit/quickwit-metastore/src/tests/index.rs @@ -25,13 +25,12 @@ // - list_indexes // - delete_index -use std::collections::BTreeSet; - use quickwit_common::rand::append_random_suffix; +use quickwit_config::merge_policy_config::{MergePolicyConfig, StableLogMergePolicyConfig}; use quickwit_config::{ - IndexConfig, RetentionPolicy, SearchSettings, SourceConfig, CLI_SOURCE_ID, INGEST_V2_SOURCE_ID, + IndexConfig, IndexingSettings, RetentionPolicy, SearchSettings, SourceConfig, CLI_SOURCE_ID, + INGEST_V2_SOURCE_ID, }; -use quickwit_doc_mapper::FieldMappingType; use quickwit_proto::metastore::{ CreateIndexRequest, DeleteIndexRequest, EntityKind, IndexMetadataFailure, IndexMetadataFailureReason, IndexMetadataRequest, IndexMetadataSubrequest, @@ -86,15 +85,14 @@ pub async fn test_metastore_create_index< cleanup_index(&mut metastore, index_uid).await; } -pub async fn test_metastore_update_index< +async fn setup_metastore_for_update< MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, ->() { - let mut metastore = MetastoreToTest::default_for_test().await; +>() -> (MetastoreToTest, IndexUid, IndexConfig) { + let metastore = MetastoreToTest::default_for_test().await; let index_id = append_random_suffix("test-update-index"); let index_uri = format!("ram:///indexes/{index_id}"); let index_config = IndexConfig::for_test(&index_id, &index_uri); - let create_index_request = CreateIndexRequest::try_from_index_config(&index_config).unwrap(); let index_uid = metastore .create_index(create_index_request.clone()) @@ -103,51 +101,31 @@ pub async fn test_metastore_update_index< .index_uid() .clone(); - let index_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) - .await - .unwrap() - .deserialize_index_metadata() - .unwrap(); - - // use all fields that are currently not set as default - let current_defaults = BTreeSet::from_iter( - index_metadata - .index_config - .search_settings - .default_search_fields, - ); - let new_search_setting = SearchSettings { - default_search_fields: index_metadata - .index_config - .doc_mapping - .field_mappings - .iter() - .filter(|f| matches!(f.mapping_type, FieldMappingType::Text(..))) - .filter(|f| !current_defaults.contains(&f.name)) - .map(|f| f.name.clone()) - .collect(), - }; + (metastore, index_uid, index_config) +} +pub async fn test_metastore_update_retention_policy< + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, +>() { + let (mut metastore, index_uid, index_config) = + setup_metastore_for_update::().await; let new_retention_policy_opt = Some(RetentionPolicy { retention_period: String::from("3 days"), evaluation_schedule: String::from("daily"), }); - assert_ne!( - index_metadata.index_config.retention_policy_opt, new_retention_policy_opt, - "original and updated value are the same, test became inefficient" - ); - // run same update twice to check idempotence, then None as a corner case check + // set and unset retention policy multiple times for loop_retention_policy_opt in [ + None, new_retention_policy_opt.clone(), new_retention_policy_opt.clone(), None, ] { let index_update = UpdateIndexRequest::try_from_updates( index_uid.clone(), - &new_search_setting, + &index_config.search_settings, &loop_retention_policy_opt, + &index_config.indexing_settings, ) .unwrap(); let response_metadata = metastore @@ -157,23 +135,124 @@ pub async fn test_metastore_update_index< .deserialize_index_metadata() .unwrap(); assert_eq!(response_metadata.index_uid, index_uid); - assert_eq!( - response_metadata.index_config.search_settings, - new_search_setting - ); assert_eq!( response_metadata.index_config.retention_policy_opt, loop_retention_policy_opt ); let updated_metadata = metastore - .index_metadata(IndexMetadataRequest::for_index_id(index_id.to_string())) + .index_metadata(IndexMetadataRequest::for_index_id( + index_uid.index_id.to_string(), + )) .await .unwrap() .deserialize_index_metadata() .unwrap(); assert_eq!(response_metadata, updated_metadata); } + cleanup_index(&mut metastore, index_uid).await; +} + +pub async fn test_metastore_update_search_settings< + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, +>() { + let (mut metastore, index_uid, index_config) = + setup_metastore_for_update::().await; + + for loop_search_settings in [ + Vec::new(), + vec!["body".to_string()], + vec!["body".to_string()], + vec!["body".to_string(), "owner".to_string()], + Vec::new(), + ] { + let index_update = UpdateIndexRequest::try_from_updates( + index_uid.clone(), + &SearchSettings { + default_search_fields: loop_search_settings.clone(), + }, + &index_config.retention_policy_opt, + &index_config.indexing_settings, + ) + .unwrap(); + let response_metadata = metastore + .update_index(index_update) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert_eq!( + response_metadata + .index_config + .search_settings + .default_search_fields, + loop_search_settings + ); + let updated_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id( + index_uid.index_id.to_string(), + )) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert_eq!( + updated_metadata + .index_config + .search_settings + .default_search_fields, + loop_search_settings + ); + } + cleanup_index(&mut metastore, index_uid).await; +} +pub async fn test_metastore_update_indexing_settings< + MetastoreToTest: MetastoreService + MetastoreServiceExt + DefaultForTest, +>() { + let (mut metastore, index_uid, index_config) = + setup_metastore_for_update::().await; + + for loop_indexing_settings in [ + MergePolicyConfig::Nop, + MergePolicyConfig::Nop, + MergePolicyConfig::StableLog(StableLogMergePolicyConfig { + merge_factor: 5, + ..Default::default() + }), + ] { + let index_update = UpdateIndexRequest::try_from_updates( + index_uid.clone(), + &index_config.search_settings, + &index_config.retention_policy_opt, + &IndexingSettings { + merge_policy: loop_indexing_settings.clone(), + ..Default::default() + }, + ) + .unwrap(); + let resp_metadata = metastore + .update_index(index_update) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert_eq!( + resp_metadata.index_config.indexing_settings.merge_policy, + loop_indexing_settings + ); + let updated_metadata = metastore + .index_metadata(IndexMetadataRequest::for_index_id( + index_uid.index_id.to_string(), + )) + .await + .unwrap() + .deserialize_index_metadata() + .unwrap(); + assert_eq!( + updated_metadata.index_config.indexing_settings.merge_policy, + loop_indexing_settings + ); + } cleanup_index(&mut metastore, index_uid).await; } diff --git a/quickwit/quickwit-metastore/src/tests/list_splits.rs b/quickwit/quickwit-metastore/src/tests/list_splits.rs index a97bdcebac9..e2955db2fc0 100644 --- a/quickwit/quickwit-metastore/src/tests/list_splits.rs +++ b/quickwit/quickwit-metastore/src/tests/list_splits.rs @@ -167,7 +167,7 @@ pub async fn test_metastore_list_all_splits< } pub async fn test_metastore_stream_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; + let metastore = MetastoreToTest::default_for_test().await; let index_id = append_random_suffix("test-stream-splits"); let index_uri = format!("ram:///indexes/{index_id}"); @@ -901,7 +901,7 @@ pub async fn test_metastore_list_splits() { - let mut metastore = MetastoreToTest::default_for_test().await; + let metastore = MetastoreToTest::default_for_test().await; let current_timestamp = OffsetDateTime::now_utc().unix_timestamp(); let index_id = append_random_suffix("test-list-splits-by-node-id"); diff --git a/quickwit/quickwit-metastore/src/tests/mod.rs b/quickwit/quickwit-metastore/src/tests/mod.rs index a8d6820548b..4a579f12eac 100644 --- a/quickwit/quickwit-metastore/src/tests/mod.rs +++ b/quickwit/quickwit-metastore/src/tests/mod.rs @@ -195,9 +195,21 @@ macro_rules! metastore_test_suite { } #[tokio::test] - async fn test_metastore_update_index() { + async fn test_metastore_update_retention_policy() { let _ = tracing_subscriber::fmt::try_init(); - $crate::tests::index::test_metastore_update_index::<$metastore_type>().await; + $crate::tests::index::test_metastore_update_retention_policy::<$metastore_type>().await; + } + + #[tokio::test] + async fn test_metastore_update_search_settings() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::index::test_metastore_update_search_settings::<$metastore_type>().await; + } + + #[tokio::test] + async fn test_metastore_update_indexing_settings() { + let _ = tracing_subscriber::fmt::try_init(); + $crate::tests::index::test_metastore_update_indexing_settings::<$metastore_type>().await; } #[tokio::test] diff --git a/quickwit/quickwit-metastore/src/tests/shard.rs b/quickwit/quickwit-metastore/src/tests/shard.rs index ec5e251fc42..b40b43d5525 100644 --- a/quickwit/quickwit-metastore/src/tests/shard.rs +++ b/quickwit/quickwit-metastore/src/tests/shard.rs @@ -35,12 +35,7 @@ use crate::{AddSourceRequestExt, CreateIndexRequestExt, MetastoreServiceExt}; #[async_trait] pub trait ReadWriteShardsForTest { - async fn insert_shards( - &mut self, - index_uid: &IndexUid, - source_id: &SourceId, - shards: Vec, - ); + async fn insert_shards(&self, index_uid: &IndexUid, source_id: &SourceId, shards: Vec); async fn list_all_shards(&self, index_uid: &IndexUid, source_id: &SourceId) -> Vec; } diff --git a/quickwit/quickwit-metastore/src/tests/split.rs b/quickwit/quickwit-metastore/src/tests/split.rs index e1590d827ed..67ee925ab6a 100644 --- a/quickwit/quickwit-metastore/src/tests/split.rs +++ b/quickwit/quickwit-metastore/src/tests/split.rs @@ -695,7 +695,7 @@ pub async fn test_metastore_publish_splits_concurrency< let mut join_handles = Vec::with_capacity(10); for partition_id in 0..10 { - let mut metastore_clone = metastore.clone(); + let metastore_clone = metastore.clone(); let index_id = index_id.clone(); let source_id = source_id.clone(); diff --git a/quickwit/quickwit-opentelemetry/src/otlp/logs.rs b/quickwit/quickwit-opentelemetry/src/otlp/logs.rs index f803bcca1ad..e5b527ac0f2 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/logs.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/logs.rs @@ -570,7 +570,7 @@ mod tests { #[tokio::test] async fn test_create_index() { - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_config = OtlpGrpcLogsService::index_config(&Uri::for_test("ram:///indexes")).unwrap(); let create_index_request = diff --git a/quickwit/quickwit-opentelemetry/src/otlp/traces.rs b/quickwit/quickwit-opentelemetry/src/otlp/traces.rs index f4ed43fd0b0..c3c4e5e1e72 100644 --- a/quickwit/quickwit-opentelemetry/src/otlp/traces.rs +++ b/quickwit/quickwit-opentelemetry/src/otlp/traces.rs @@ -925,7 +925,7 @@ mod tests { #[tokio::test] async fn test_create_index() { - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_config = OtlpGrpcTracesService::index_config(&Uri::for_test("ram:///indexes")).unwrap(); let create_index_request = diff --git a/quickwit/quickwit-proto/Cargo.toml b/quickwit/quickwit-proto/Cargo.toml index b7d8979590d..8ba844df054 100644 --- a/quickwit/quickwit-proto/Cargo.toml +++ b/quickwit/quickwit-proto/Cargo.toml @@ -15,7 +15,6 @@ async-trait = { workspace = true } bytes = { workspace = true } bytesize = { workspace = true } bytestring = { workspace = true } -dyn-clone = { workspace = true } futures = { workspace = true, optional = true } http = { workspace = true } mockall = { workspace = true, optional = true } diff --git a/quickwit/quickwit-proto/protos/quickwit/metastore.proto b/quickwit/quickwit-proto/protos/quickwit/metastore.proto index ac4e35b0817..084dccad417 100644 --- a/quickwit/quickwit-proto/protos/quickwit/metastore.proto +++ b/quickwit/quickwit-proto/protos/quickwit/metastore.proto @@ -215,6 +215,7 @@ message UpdateIndexRequest { quickwit.common.IndexUid index_uid = 1; string search_settings_json = 2; optional string retention_policy_json = 3; + string indexing_settings_json = 4; } message ListIndexesMetadataRequest { diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs index 2a459531903..38471f0ad7b 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.cluster.rs @@ -94,23 +94,18 @@ impl RpcName for FetchClusterStateRequest { } #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait ClusterService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait ClusterService: std::fmt::Debug + Send + Sync + 'static { async fn fetch_cluster_state( - &mut self, + &self, request: FetchClusterStateRequest, ) -> crate::cluster::ClusterResult; } -dyn_clone::clone_trait_object!(ClusterService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockClusterService { - fn clone(&self) -> Self { - MockClusterService::new() - } -} #[derive(Debug, Clone)] pub struct ClusterServiceClient { - inner: Box, + inner: InnerClusterServiceClient, } +#[derive(Debug, Clone)] +struct InnerClusterServiceClient(std::sync::Arc); impl ClusterServiceClient { pub fn new(instance: T) -> Self where @@ -122,7 +117,9 @@ impl ClusterServiceClient { MockClusterService > (), "`MockClusterService` must be wrapped in a `MockClusterServiceWrapper`: use `ClusterServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerClusterServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -181,7 +178,7 @@ impl ClusterServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockClusterService) -> Self { let mock_wrapper = mock_cluster_service::MockClusterServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -193,23 +190,23 @@ impl ClusterServiceClient { #[async_trait::async_trait] impl ClusterService for ClusterServiceClient { async fn fetch_cluster_state( - &mut self, + &self, request: FetchClusterStateRequest, ) -> crate::cluster::ClusterResult { - self.inner.fetch_cluster_state(request).await + self.inner.0.fetch_cluster_state(request).await } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_cluster_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockClusterServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl ClusterService for MockClusterServiceWrapper { async fn fetch_cluster_state( - &mut self, + &self, request: super::FetchClusterStateRequest, ) -> crate::cluster::ClusterResult { self.inner.lock().await.fetch_cluster_state(request).await @@ -219,7 +216,7 @@ pub mod mock_cluster_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerClusterServiceClient { type Response = FetchClusterStateResponse; type Error = crate::cluster::ClusterError; type Future = BoxFuture; @@ -230,36 +227,29 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: FetchClusterStateRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.fetch_cluster_state(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.fetch_cluster_state(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct ClusterServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerClusterServiceClient, fetch_cluster_state_svc: quickwit_common::tower::BoxService< FetchClusterStateRequest, FetchClusterStateResponse, crate::cluster::ClusterError, >, } -impl Clone for ClusterServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - fetch_cluster_state_svc: self.fetch_cluster_state_svc.clone(), - } - } -} #[async_trait::async_trait] impl ClusterService for ClusterServiceTowerServiceStack { async fn fetch_cluster_state( - &mut self, + &self, request: FetchClusterStateRequest, ) -> crate::cluster::ClusterResult { - self.fetch_cluster_state_svc.ready().await?.call(request).await + self.fetch_cluster_state_svc.clone().ready().await?.call(request).await } } type FetchClusterStateLayer = quickwit_common::tower::BoxLayer< @@ -333,7 +323,8 @@ impl ClusterServiceTowerLayerStack { where T: ClusterService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerClusterServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -341,23 +332,21 @@ impl ClusterServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> ClusterServiceClient { - self.build_from_boxed( - Box::new(ClusterServiceClient::from_channel(addr, channel, max_message_size)), - ) + let client = ClusterServiceClient::from_channel(addr, channel, max_message_size); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> ClusterServiceClient { - self.build_from_boxed( - Box::new( - ClusterServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = ClusterServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -367,26 +356,31 @@ impl ClusterServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, ClusterServiceMailbox: ClusterService, { - self.build_from_boxed(Box::new(ClusterServiceMailbox::new(mailbox))) + let inner_client = InnerClusterServiceClient( + std::sync::Arc::new(ClusterServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: MockClusterService) -> ClusterServiceClient { - self.build_from_boxed(Box::new(ClusterServiceClient::from_mock(mock))) + let client = ClusterServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerClusterServiceClient, ) -> ClusterServiceClient { let fetch_cluster_state_svc = self .fetch_cluster_state_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = ClusterServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, fetch_cluster_state_svc, }; ClusterServiceClient::new(tower_svc_stack) @@ -472,10 +466,10 @@ where >, { async fn fetch_cluster_state( - &mut self, + &self, request: FetchClusterStateRequest, ) -> crate::cluster::ClusterResult { - self.call(request).await + self.clone().call(request).await } } #[derive(Debug, Clone)] @@ -513,10 +507,11 @@ where T::Future: Send, { async fn fetch_cluster_state( - &mut self, + &self, request: FetchClusterStateRequest, ) -> crate::cluster::ClusterResult { self.inner + .clone() .fetch_cluster_state(request) .await .map(|response| response.into_inner()) @@ -528,14 +523,16 @@ where } #[derive(Debug)] pub struct ClusterServiceGrpcServerAdapter { - inner: Box, + inner: InnerClusterServiceClient, } impl ClusterServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: ClusterService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerClusterServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -546,7 +543,7 @@ for ClusterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .fetch_cluster_state(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs index 2fb173e079a..29d91cf6de0 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.control_plane.rs @@ -131,62 +131,57 @@ use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait ControlPlaneService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait ControlPlaneService: std::fmt::Debug + Send + Sync + 'static { /// Creates a new index. async fn create_index( - &mut self, + &self, request: super::metastore::CreateIndexRequest, ) -> crate::control_plane::ControlPlaneResult; /// Updates an index. async fn update_index( - &mut self, + &self, request: super::metastore::UpdateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::IndexMetadataResponse, >; /// Deletes an index. async fn delete_index( - &mut self, + &self, request: super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult; /// Adds a source to an index. async fn add_source( - &mut self, + &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult; /// Enables or disables a source. async fn toggle_source( - &mut self, + &self, request: super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult; /// Removes a source from an index. async fn delete_source( - &mut self, + &self, request: super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult; /// Returns the list of open shards for one or several sources. If the control plane is not able to find any /// for a source, it will pick a pair of leader-follower ingesters and will open a new shard. async fn get_or_create_open_shards( - &mut self, + &self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult; /// Asks the control plane whether the shards listed in the request should be deleted or truncated. async fn advise_reset_shards( - &mut self, + &self, request: AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult; } -dyn_clone::clone_trait_object!(ControlPlaneService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockControlPlaneService { - fn clone(&self) -> Self { - MockControlPlaneService::new() - } -} #[derive(Debug, Clone)] pub struct ControlPlaneServiceClient { - inner: Box, + inner: InnerControlPlaneServiceClient, } +#[derive(Debug, Clone)] +struct InnerControlPlaneServiceClient(std::sync::Arc); impl ControlPlaneServiceClient { pub fn new(instance: T) -> Self where @@ -198,7 +193,9 @@ impl ControlPlaneServiceClient { MockControlPlaneService > (), "`MockControlPlaneService` must be wrapped in a `MockControlPlaneServiceWrapper`: use `ControlPlaneServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerControlPlaneServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -259,7 +256,7 @@ impl ControlPlaneServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockControlPlaneService) -> Self { let mock_wrapper = mock_control_plane_service::MockControlPlaneServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -271,69 +268,69 @@ impl ControlPlaneServiceClient { #[async_trait::async_trait] impl ControlPlaneService for ControlPlaneServiceClient { async fn create_index( - &mut self, + &self, request: super::metastore::CreateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::CreateIndexResponse, > { - self.inner.create_index(request).await + self.inner.0.create_index(request).await } async fn update_index( - &mut self, + &self, request: super::metastore::UpdateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::IndexMetadataResponse, > { - self.inner.update_index(request).await + self.inner.0.update_index(request).await } async fn delete_index( - &mut self, + &self, request: super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult { - self.inner.delete_index(request).await + self.inner.0.delete_index(request).await } async fn add_source( - &mut self, + &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.inner.add_source(request).await + self.inner.0.add_source(request).await } async fn toggle_source( - &mut self, + &self, request: super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.inner.toggle_source(request).await + self.inner.0.toggle_source(request).await } async fn delete_source( - &mut self, + &self, request: super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.inner.delete_source(request).await + self.inner.0.delete_source(request).await } async fn get_or_create_open_shards( - &mut self, + &self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult { - self.inner.get_or_create_open_shards(request).await + self.inner.0.get_or_create_open_shards(request).await } async fn advise_reset_shards( - &mut self, + &self, request: AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult { - self.inner.advise_reset_shards(request).await + self.inner.0.advise_reset_shards(request).await } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_control_plane_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockControlPlaneServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl ControlPlaneService for MockControlPlaneServiceWrapper { async fn create_index( - &mut self, + &self, request: super::super::metastore::CreateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::super::metastore::CreateIndexResponse, @@ -341,7 +338,7 @@ pub mod mock_control_plane_service { self.inner.lock().await.create_index(request).await } async fn update_index( - &mut self, + &self, request: super::super::metastore::UpdateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::super::metastore::IndexMetadataResponse, @@ -349,7 +346,7 @@ pub mod mock_control_plane_service { self.inner.lock().await.update_index(request).await } async fn delete_index( - &mut self, + &self, request: super::super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::super::metastore::EmptyResponse, @@ -357,7 +354,7 @@ pub mod mock_control_plane_service { self.inner.lock().await.delete_index(request).await } async fn add_source( - &mut self, + &self, request: super::super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult< super::super::metastore::EmptyResponse, @@ -365,7 +362,7 @@ pub mod mock_control_plane_service { self.inner.lock().await.add_source(request).await } async fn toggle_source( - &mut self, + &self, request: super::super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult< super::super::metastore::EmptyResponse, @@ -373,7 +370,7 @@ pub mod mock_control_plane_service { self.inner.lock().await.toggle_source(request).await } async fn delete_source( - &mut self, + &self, request: super::super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult< super::super::metastore::EmptyResponse, @@ -381,7 +378,7 @@ pub mod mock_control_plane_service { self.inner.lock().await.delete_source(request).await } async fn get_or_create_open_shards( - &mut self, + &self, request: super::GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult< super::GetOrCreateOpenShardsResponse, @@ -389,7 +386,7 @@ pub mod mock_control_plane_service { self.inner.lock().await.get_or_create_open_shards(request).await } async fn advise_reset_shards( - &mut self, + &self, request: super::AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner.lock().await.advise_reset_shards(request).await @@ -400,7 +397,7 @@ pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; impl tower::Service -for Box { +for InnerControlPlaneServiceClient { type Response = super::metastore::CreateIndexResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -411,13 +408,13 @@ for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: super::metastore::CreateIndexRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.create_index(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.create_index(request).await }; Box::pin(fut) } } impl tower::Service -for Box { +for InnerControlPlaneServiceClient { type Response = super::metastore::IndexMetadataResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -428,13 +425,13 @@ for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: super::metastore::UpdateIndexRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.update_index(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.update_index(request).await }; Box::pin(fut) } } impl tower::Service -for Box { +for InnerControlPlaneServiceClient { type Response = super::metastore::EmptyResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -445,13 +442,13 @@ for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: super::metastore::DeleteIndexRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_index(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.delete_index(request).await }; Box::pin(fut) } } impl tower::Service -for Box { +for InnerControlPlaneServiceClient { type Response = super::metastore::EmptyResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -462,13 +459,13 @@ for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: super::metastore::AddSourceRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.add_source(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.add_source(request).await }; Box::pin(fut) } } impl tower::Service -for Box { +for InnerControlPlaneServiceClient { type Response = super::metastore::EmptyResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -479,13 +476,13 @@ for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: super::metastore::ToggleSourceRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.toggle_source(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.toggle_source(request).await }; Box::pin(fut) } } impl tower::Service -for Box { +for InnerControlPlaneServiceClient { type Response = super::metastore::EmptyResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -496,12 +493,12 @@ for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: super::metastore::DeleteSourceRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_source(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.delete_source(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerControlPlaneServiceClient { type Response = GetOrCreateOpenShardsResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -512,12 +509,12 @@ impl tower::Service for Box Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.get_or_create_open_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.get_or_create_open_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerControlPlaneServiceClient { type Response = AdviseResetShardsResponse; type Error = crate::control_plane::ControlPlaneError; type Future = BoxFuture; @@ -528,15 +525,16 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: AdviseResetShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.advise_reset_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.advise_reset_shards(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct ControlPlaneServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerControlPlaneServiceClient, create_index_svc: quickwit_common::tower::BoxService< super::metastore::CreateIndexRequest, super::metastore::CreateIndexResponse, @@ -578,74 +576,59 @@ struct ControlPlaneServiceTowerServiceStack { crate::control_plane::ControlPlaneError, >, } -impl Clone for ControlPlaneServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - create_index_svc: self.create_index_svc.clone(), - update_index_svc: self.update_index_svc.clone(), - delete_index_svc: self.delete_index_svc.clone(), - add_source_svc: self.add_source_svc.clone(), - toggle_source_svc: self.toggle_source_svc.clone(), - delete_source_svc: self.delete_source_svc.clone(), - get_or_create_open_shards_svc: self.get_or_create_open_shards_svc.clone(), - advise_reset_shards_svc: self.advise_reset_shards_svc.clone(), - } - } -} #[async_trait::async_trait] impl ControlPlaneService for ControlPlaneServiceTowerServiceStack { async fn create_index( - &mut self, + &self, request: super::metastore::CreateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::CreateIndexResponse, > { - self.create_index_svc.ready().await?.call(request).await + self.create_index_svc.clone().ready().await?.call(request).await } async fn update_index( - &mut self, + &self, request: super::metastore::UpdateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::IndexMetadataResponse, > { - self.update_index_svc.ready().await?.call(request).await + self.update_index_svc.clone().ready().await?.call(request).await } async fn delete_index( - &mut self, + &self, request: super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult { - self.delete_index_svc.ready().await?.call(request).await + self.delete_index_svc.clone().ready().await?.call(request).await } async fn add_source( - &mut self, + &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.add_source_svc.ready().await?.call(request).await + self.add_source_svc.clone().ready().await?.call(request).await } async fn toggle_source( - &mut self, + &self, request: super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.toggle_source_svc.ready().await?.call(request).await + self.toggle_source_svc.clone().ready().await?.call(request).await } async fn delete_source( - &mut self, + &self, request: super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.delete_source_svc.ready().await?.call(request).await + self.delete_source_svc.clone().ready().await?.call(request).await } async fn get_or_create_open_shards( - &mut self, + &self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult { - self.get_or_create_open_shards_svc.ready().await?.call(request).await + self.get_or_create_open_shards_svc.clone().ready().await?.call(request).await } async fn advise_reset_shards( - &mut self, + &self, request: AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult { - self.advise_reset_shards_svc.ready().await?.call(request).await + self.advise_reset_shards_svc.clone().ready().await?.call(request).await } } type CreateIndexLayer = quickwit_common::tower::BoxLayer< @@ -1147,7 +1130,8 @@ impl ControlPlaneServiceTowerLayerStack { where T: ControlPlaneService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerControlPlaneServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -1155,25 +1139,25 @@ impl ControlPlaneServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> ControlPlaneServiceClient { - self.build_from_boxed( - Box::new( - ControlPlaneServiceClient::from_channel(addr, channel, max_message_size), - ), - ) + let client = ControlPlaneServiceClient::from_channel( + addr, + channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> ControlPlaneServiceClient { - self.build_from_boxed( - Box::new( - ControlPlaneServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = ControlPlaneServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -1183,25 +1167,30 @@ impl ControlPlaneServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, ControlPlaneServiceMailbox: ControlPlaneService, { - self.build_from_boxed(Box::new(ControlPlaneServiceMailbox::new(mailbox))) + let inner_client = InnerControlPlaneServiceClient( + std::sync::Arc::new(ControlPlaneServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock( self, mock: MockControlPlaneService, ) -> ControlPlaneServiceClient { - self.build_from_boxed(Box::new(ControlPlaneServiceClient::from_mock(mock))) + let client = ControlPlaneServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerControlPlaneServiceClient, ) -> ControlPlaneServiceClient { let create_index_svc = self .create_index_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let update_index_svc = self @@ -1209,7 +1198,7 @@ impl ControlPlaneServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let delete_index_svc = self @@ -1217,7 +1206,7 @@ impl ControlPlaneServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let add_source_svc = self @@ -1225,7 +1214,7 @@ impl ControlPlaneServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let toggle_source_svc = self @@ -1233,7 +1222,7 @@ impl ControlPlaneServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let delete_source_svc = self @@ -1241,7 +1230,7 @@ impl ControlPlaneServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let get_or_create_open_shards_svc = self @@ -1249,7 +1238,7 @@ impl ControlPlaneServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let advise_reset_shards_svc = self @@ -1257,11 +1246,11 @@ impl ControlPlaneServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = ControlPlaneServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, create_index_svc, update_index_svc, delete_index_svc, @@ -1420,56 +1409,56 @@ where >, { async fn create_index( - &mut self, + &self, request: super::metastore::CreateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::CreateIndexResponse, > { - self.call(request).await + self.clone().call(request).await } async fn update_index( - &mut self, + &self, request: super::metastore::UpdateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::IndexMetadataResponse, > { - self.call(request).await + self.clone().call(request).await } async fn delete_index( - &mut self, + &self, request: super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await + self.clone().call(request).await } async fn add_source( - &mut self, + &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await + self.clone().call(request).await } async fn toggle_source( - &mut self, + &self, request: super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await + self.clone().call(request).await } async fn delete_source( - &mut self, + &self, request: super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await + self.clone().call(request).await } async fn get_or_create_open_shards( - &mut self, + &self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await + self.clone().call(request).await } async fn advise_reset_shards( - &mut self, + &self, request: AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult { - self.call(request).await + self.clone().call(request).await } } #[derive(Debug, Clone)] @@ -1507,12 +1496,13 @@ where T::Future: Send, { async fn create_index( - &mut self, + &self, request: super::metastore::CreateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::CreateIndexResponse, > { self.inner + .clone() .create_index(request) .await .map(|response| response.into_inner()) @@ -1522,12 +1512,13 @@ where )) } async fn update_index( - &mut self, + &self, request: super::metastore::UpdateIndexRequest, ) -> crate::control_plane::ControlPlaneResult< super::metastore::IndexMetadataResponse, > { self.inner + .clone() .update_index(request) .await .map(|response| response.into_inner()) @@ -1537,10 +1528,11 @@ where )) } async fn delete_index( - &mut self, + &self, request: super::metastore::DeleteIndexRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner + .clone() .delete_index(request) .await .map(|response| response.into_inner()) @@ -1550,10 +1542,11 @@ where )) } async fn add_source( - &mut self, + &self, request: super::metastore::AddSourceRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner + .clone() .add_source(request) .await .map(|response| response.into_inner()) @@ -1563,10 +1556,11 @@ where )) } async fn toggle_source( - &mut self, + &self, request: super::metastore::ToggleSourceRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner + .clone() .toggle_source(request) .await .map(|response| response.into_inner()) @@ -1576,10 +1570,11 @@ where )) } async fn delete_source( - &mut self, + &self, request: super::metastore::DeleteSourceRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner + .clone() .delete_source(request) .await .map(|response| response.into_inner()) @@ -1589,10 +1584,11 @@ where )) } async fn get_or_create_open_shards( - &mut self, + &self, request: GetOrCreateOpenShardsRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner + .clone() .get_or_create_open_shards(request) .await .map(|response| response.into_inner()) @@ -1602,10 +1598,11 @@ where )) } async fn advise_reset_shards( - &mut self, + &self, request: AdviseResetShardsRequest, ) -> crate::control_plane::ControlPlaneResult { self.inner + .clone() .advise_reset_shards(request) .await .map(|response| response.into_inner()) @@ -1617,14 +1614,16 @@ where } #[derive(Debug)] pub struct ControlPlaneServiceGrpcServerAdapter { - inner: Box, + inner: InnerControlPlaneServiceClient, } impl ControlPlaneServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: ControlPlaneService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerControlPlaneServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -1635,7 +1634,7 @@ for ControlPlaneServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .create_index(request.into_inner()) .await .map(tonic::Response::new) @@ -1649,7 +1648,7 @@ for ControlPlaneServiceGrpcServerAdapter { tonic::Status, > { self.inner - .clone() + .0 .update_index(request.into_inner()) .await .map(tonic::Response::new) @@ -1660,7 +1659,7 @@ for ControlPlaneServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .delete_index(request.into_inner()) .await .map(tonic::Response::new) @@ -1671,7 +1670,7 @@ for ControlPlaneServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .add_source(request.into_inner()) .await .map(tonic::Response::new) @@ -1682,7 +1681,7 @@ for ControlPlaneServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .toggle_source(request.into_inner()) .await .map(tonic::Response::new) @@ -1693,7 +1692,7 @@ for ControlPlaneServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .delete_source(request.into_inner()) .await .map(tonic::Response::new) @@ -1704,7 +1703,7 @@ for ControlPlaneServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .get_or_create_open_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -1715,7 +1714,7 @@ for ControlPlaneServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .advise_reset_shards(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs index 4ec5d94a9e8..b05cc01aef8 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.developer.rs @@ -25,23 +25,18 @@ impl RpcName for GetDebugInfoRequest { } #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait DeveloperService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait DeveloperService: std::fmt::Debug + Send + Sync + 'static { async fn get_debug_info( - &mut self, + &self, request: GetDebugInfoRequest, ) -> crate::developer::DeveloperResult; } -dyn_clone::clone_trait_object!(DeveloperService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockDeveloperService { - fn clone(&self) -> Self { - MockDeveloperService::new() - } -} #[derive(Debug, Clone)] pub struct DeveloperServiceClient { - inner: Box, + inner: InnerDeveloperServiceClient, } +#[derive(Debug, Clone)] +struct InnerDeveloperServiceClient(std::sync::Arc); impl DeveloperServiceClient { pub fn new(instance: T) -> Self where @@ -53,7 +48,9 @@ impl DeveloperServiceClient { MockDeveloperService > (), "`MockDeveloperService` must be wrapped in a `MockDeveloperServiceWrapper`: use `DeveloperServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerDeveloperServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -114,7 +111,7 @@ impl DeveloperServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockDeveloperService) -> Self { let mock_wrapper = mock_developer_service::MockDeveloperServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -126,23 +123,23 @@ impl DeveloperServiceClient { #[async_trait::async_trait] impl DeveloperService for DeveloperServiceClient { async fn get_debug_info( - &mut self, + &self, request: GetDebugInfoRequest, ) -> crate::developer::DeveloperResult { - self.inner.get_debug_info(request).await + self.inner.0.get_debug_info(request).await } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_developer_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockDeveloperServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl DeveloperService for MockDeveloperServiceWrapper { async fn get_debug_info( - &mut self, + &self, request: super::GetDebugInfoRequest, ) -> crate::developer::DeveloperResult { self.inner.lock().await.get_debug_info(request).await @@ -152,7 +149,7 @@ pub mod mock_developer_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerDeveloperServiceClient { type Response = GetDebugInfoResponse; type Error = crate::developer::DeveloperError; type Future = BoxFuture; @@ -163,36 +160,29 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: GetDebugInfoRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.get_debug_info(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.get_debug_info(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct DeveloperServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerDeveloperServiceClient, get_debug_info_svc: quickwit_common::tower::BoxService< GetDebugInfoRequest, GetDebugInfoResponse, crate::developer::DeveloperError, >, } -impl Clone for DeveloperServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - get_debug_info_svc: self.get_debug_info_svc.clone(), - } - } -} #[async_trait::async_trait] impl DeveloperService for DeveloperServiceTowerServiceStack { async fn get_debug_info( - &mut self, + &self, request: GetDebugInfoRequest, ) -> crate::developer::DeveloperResult { - self.get_debug_info_svc.ready().await?.call(request).await + self.get_debug_info_svc.clone().ready().await?.call(request).await } } type GetDebugInfoLayer = quickwit_common::tower::BoxLayer< @@ -265,7 +255,8 @@ impl DeveloperServiceTowerLayerStack { where T: DeveloperService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerDeveloperServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -273,25 +264,25 @@ impl DeveloperServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> DeveloperServiceClient { - self.build_from_boxed( - Box::new( - DeveloperServiceClient::from_channel(addr, channel, max_message_size), - ), - ) + let client = DeveloperServiceClient::from_channel( + addr, + channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> DeveloperServiceClient { - self.build_from_boxed( - Box::new( - DeveloperServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = DeveloperServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -301,26 +292,31 @@ impl DeveloperServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, DeveloperServiceMailbox: DeveloperService, { - self.build_from_boxed(Box::new(DeveloperServiceMailbox::new(mailbox))) + let inner_client = InnerDeveloperServiceClient( + std::sync::Arc::new(DeveloperServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: MockDeveloperService) -> DeveloperServiceClient { - self.build_from_boxed(Box::new(DeveloperServiceClient::from_mock(mock))) + let client = DeveloperServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerDeveloperServiceClient, ) -> DeveloperServiceClient { let get_debug_info_svc = self .get_debug_info_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = DeveloperServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, get_debug_info_svc, }; DeveloperServiceClient::new(tower_svc_stack) @@ -406,10 +402,10 @@ where >, { async fn get_debug_info( - &mut self, + &self, request: GetDebugInfoRequest, ) -> crate::developer::DeveloperResult { - self.call(request).await + self.clone().call(request).await } } #[derive(Debug, Clone)] @@ -447,10 +443,11 @@ where T::Future: Send, { async fn get_debug_info( - &mut self, + &self, request: GetDebugInfoRequest, ) -> crate::developer::DeveloperResult { self.inner + .clone() .get_debug_info(request) .await .map(|response| response.into_inner()) @@ -462,14 +459,16 @@ where } #[derive(Debug)] pub struct DeveloperServiceGrpcServerAdapter { - inner: Box, + inner: InnerDeveloperServiceClient, } impl DeveloperServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: DeveloperService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerDeveloperServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -480,7 +479,7 @@ for DeveloperServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .get_debug_info(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs index 5f569509566..60a31926f60 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.indexing.rs @@ -32,24 +32,19 @@ use std::str::FromStr; use tower::{Layer, Service, ServiceExt}; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait IndexingService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait IndexingService: std::fmt::Debug + Send + Sync + 'static { /// Apply an indexing plan on the node. async fn apply_indexing_plan( - &mut self, + &self, request: ApplyIndexingPlanRequest, ) -> crate::indexing::IndexingResult; } -dyn_clone::clone_trait_object!(IndexingService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockIndexingService { - fn clone(&self) -> Self { - MockIndexingService::new() - } -} #[derive(Debug, Clone)] pub struct IndexingServiceClient { - inner: Box, + inner: InnerIndexingServiceClient, } +#[derive(Debug, Clone)] +struct InnerIndexingServiceClient(std::sync::Arc); impl IndexingServiceClient { pub fn new(instance: T) -> Self where @@ -61,7 +56,9 @@ impl IndexingServiceClient { MockIndexingService > (), "`MockIndexingService` must be wrapped in a `MockIndexingServiceWrapper`: use `IndexingServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerIndexingServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -122,7 +119,7 @@ impl IndexingServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockIndexingService) -> Self { let mock_wrapper = mock_indexing_service::MockIndexingServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -134,23 +131,23 @@ impl IndexingServiceClient { #[async_trait::async_trait] impl IndexingService for IndexingServiceClient { async fn apply_indexing_plan( - &mut self, + &self, request: ApplyIndexingPlanRequest, ) -> crate::indexing::IndexingResult { - self.inner.apply_indexing_plan(request).await + self.inner.0.apply_indexing_plan(request).await } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_indexing_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockIndexingServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl IndexingService for MockIndexingServiceWrapper { async fn apply_indexing_plan( - &mut self, + &self, request: super::ApplyIndexingPlanRequest, ) -> crate::indexing::IndexingResult { self.inner.lock().await.apply_indexing_plan(request).await @@ -160,7 +157,7 @@ pub mod mock_indexing_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerIndexingServiceClient { type Response = ApplyIndexingPlanResponse; type Error = crate::indexing::IndexingError; type Future = BoxFuture; @@ -171,36 +168,29 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ApplyIndexingPlanRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.apply_indexing_plan(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.apply_indexing_plan(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct IndexingServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerIndexingServiceClient, apply_indexing_plan_svc: quickwit_common::tower::BoxService< ApplyIndexingPlanRequest, ApplyIndexingPlanResponse, crate::indexing::IndexingError, >, } -impl Clone for IndexingServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - apply_indexing_plan_svc: self.apply_indexing_plan_svc.clone(), - } - } -} #[async_trait::async_trait] impl IndexingService for IndexingServiceTowerServiceStack { async fn apply_indexing_plan( - &mut self, + &self, request: ApplyIndexingPlanRequest, ) -> crate::indexing::IndexingResult { - self.apply_indexing_plan_svc.ready().await?.call(request).await + self.apply_indexing_plan_svc.clone().ready().await?.call(request).await } } type ApplyIndexingPlanLayer = quickwit_common::tower::BoxLayer< @@ -274,7 +264,8 @@ impl IndexingServiceTowerLayerStack { where T: IndexingService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerIndexingServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -282,25 +273,25 @@ impl IndexingServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> IndexingServiceClient { - self.build_from_boxed( - Box::new( - IndexingServiceClient::from_channel(addr, channel, max_message_size), - ), - ) + let client = IndexingServiceClient::from_channel( + addr, + channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> IndexingServiceClient { - self.build_from_boxed( - Box::new( - IndexingServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = IndexingServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -310,26 +301,31 @@ impl IndexingServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, IndexingServiceMailbox: IndexingService, { - self.build_from_boxed(Box::new(IndexingServiceMailbox::new(mailbox))) + let inner_client = InnerIndexingServiceClient( + std::sync::Arc::new(IndexingServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: MockIndexingService) -> IndexingServiceClient { - self.build_from_boxed(Box::new(IndexingServiceClient::from_mock(mock))) + let client = IndexingServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerIndexingServiceClient, ) -> IndexingServiceClient { let apply_indexing_plan_svc = self .apply_indexing_plan_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = IndexingServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, apply_indexing_plan_svc, }; IndexingServiceClient::new(tower_svc_stack) @@ -415,10 +411,10 @@ where >, { async fn apply_indexing_plan( - &mut self, + &self, request: ApplyIndexingPlanRequest, ) -> crate::indexing::IndexingResult { - self.call(request).await + self.clone().call(request).await } } #[derive(Debug, Clone)] @@ -456,10 +452,11 @@ where T::Future: Send, { async fn apply_indexing_plan( - &mut self, + &self, request: ApplyIndexingPlanRequest, ) -> crate::indexing::IndexingResult { self.inner + .clone() .apply_indexing_plan(request) .await .map(|response| response.into_inner()) @@ -471,14 +468,16 @@ where } #[derive(Debug)] pub struct IndexingServiceGrpcServerAdapter { - inner: Box, + inner: InnerIndexingServiceClient, } impl IndexingServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: IndexingService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerIndexingServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -489,7 +488,7 @@ for IndexingServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .apply_indexing_plan(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs index 584a20293e5..ace0b1d63b0 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.ingester.rs @@ -611,66 +611,61 @@ pub type IngesterServiceStream = quickwit_common::ServiceStream< >; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait IngesterService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait IngesterService: std::fmt::Debug + Send + Sync + 'static { /// Persists batches of documents to primary shards hosted on a leader. async fn persist( - &mut self, + &self, request: PersistRequest, ) -> crate::ingest::IngestV2Result; /// Opens a replication stream from a leader to a follower. async fn open_replication_stream( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result>; /// Streams records from a leader or a follower. The client can optionally specify a range of positions to fetch, /// otherwise the stream will go undefinitely or until the shard is closed. async fn open_fetch_stream( - &mut self, + &self, request: OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result>; /// Streams status updates, called "observations", from an ingester. async fn open_observation_stream( - &mut self, + &self, request: OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result>; /// Creates and initializes a set of newly opened shards. This RPC is called by the control plane on leaders. async fn init_shards( - &mut self, + &self, request: InitShardsRequest, ) -> crate::ingest::IngestV2Result; /// Only retain the shards that are listed in the request. /// Other shards are deleted. async fn retain_shards( - &mut self, + &self, request: RetainShardsRequest, ) -> crate::ingest::IngestV2Result; /// Truncates a set of shards at the given positions. This RPC is called by indexers on leaders AND followers. async fn truncate_shards( - &mut self, + &self, request: TruncateShardsRequest, ) -> crate::ingest::IngestV2Result; /// Closes a set of shards. This RPC is called by the control plane. async fn close_shards( - &mut self, + &self, request: CloseShardsRequest, ) -> crate::ingest::IngestV2Result; /// Decommissions the ingester. async fn decommission( - &mut self, + &self, request: DecommissionRequest, ) -> crate::ingest::IngestV2Result; } -dyn_clone::clone_trait_object!(IngesterService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockIngesterService { - fn clone(&self) -> Self { - MockIngesterService::new() - } -} #[derive(Debug, Clone)] pub struct IngesterServiceClient { - inner: Box, + inner: InnerIngesterServiceClient, } +#[derive(Debug, Clone)] +struct InnerIngesterServiceClient(std::sync::Arc); impl IngesterServiceClient { pub fn new(instance: T) -> Self where @@ -682,7 +677,9 @@ impl IngesterServiceClient { MockIngesterService > (), "`MockIngesterService` must be wrapped in a `MockIngesterServiceWrapper`: use `IngesterServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerIngesterServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -743,7 +740,7 @@ impl IngesterServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockIngesterService) -> Self { let mock_wrapper = mock_ingester_service::MockIngesterServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -755,77 +752,77 @@ impl IngesterServiceClient { #[async_trait::async_trait] impl IngesterService for IngesterServiceClient { async fn persist( - &mut self, + &self, request: PersistRequest, ) -> crate::ingest::IngestV2Result { - self.inner.persist(request).await + self.inner.0.persist(request).await } async fn open_replication_stream( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result> { - self.inner.open_replication_stream(request).await + self.inner.0.open_replication_stream(request).await } async fn open_fetch_stream( - &mut self, + &self, request: OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result> { - self.inner.open_fetch_stream(request).await + self.inner.0.open_fetch_stream(request).await } async fn open_observation_stream( - &mut self, + &self, request: OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result> { - self.inner.open_observation_stream(request).await + self.inner.0.open_observation_stream(request).await } async fn init_shards( - &mut self, + &self, request: InitShardsRequest, ) -> crate::ingest::IngestV2Result { - self.inner.init_shards(request).await + self.inner.0.init_shards(request).await } async fn retain_shards( - &mut self, + &self, request: RetainShardsRequest, ) -> crate::ingest::IngestV2Result { - self.inner.retain_shards(request).await + self.inner.0.retain_shards(request).await } async fn truncate_shards( - &mut self, + &self, request: TruncateShardsRequest, ) -> crate::ingest::IngestV2Result { - self.inner.truncate_shards(request).await + self.inner.0.truncate_shards(request).await } async fn close_shards( - &mut self, + &self, request: CloseShardsRequest, ) -> crate::ingest::IngestV2Result { - self.inner.close_shards(request).await + self.inner.0.close_shards(request).await } async fn decommission( - &mut self, + &self, request: DecommissionRequest, ) -> crate::ingest::IngestV2Result { - self.inner.decommission(request).await + self.inner.0.decommission(request).await } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_ingester_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockIngesterServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl IngesterService for MockIngesterServiceWrapper { async fn persist( - &mut self, + &self, request: super::PersistRequest, ) -> crate::ingest::IngestV2Result { self.inner.lock().await.persist(request).await } async fn open_replication_stream( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result< IngesterServiceStream, @@ -833,13 +830,13 @@ pub mod mock_ingester_service { self.inner.lock().await.open_replication_stream(request).await } async fn open_fetch_stream( - &mut self, + &self, request: super::OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result> { self.inner.lock().await.open_fetch_stream(request).await } async fn open_observation_stream( - &mut self, + &self, request: super::OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result< IngesterServiceStream, @@ -847,31 +844,31 @@ pub mod mock_ingester_service { self.inner.lock().await.open_observation_stream(request).await } async fn init_shards( - &mut self, + &self, request: super::InitShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.lock().await.init_shards(request).await } async fn retain_shards( - &mut self, + &self, request: super::RetainShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.lock().await.retain_shards(request).await } async fn truncate_shards( - &mut self, + &self, request: super::TruncateShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.lock().await.truncate_shards(request).await } async fn close_shards( - &mut self, + &self, request: super::CloseShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner.lock().await.close_shards(request).await } async fn decommission( - &mut self, + &self, request: super::DecommissionRequest, ) -> crate::ingest::IngestV2Result { self.inner.lock().await.decommission(request).await @@ -881,7 +878,7 @@ pub mod mock_ingester_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = PersistResponse; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -892,13 +889,13 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: PersistRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.persist(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.persist(request).await }; Box::pin(fut) } } impl tower::Service> -for Box { +for InnerIngesterServiceClient { type Response = IngesterServiceStream; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -912,12 +909,12 @@ for Box { &mut self, request: quickwit_common::ServiceStream, ) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.open_replication_stream(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.open_replication_stream(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = IngesterServiceStream; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -928,12 +925,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: OpenFetchStreamRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.open_fetch_stream(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.open_fetch_stream(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = IngesterServiceStream; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -944,12 +941,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: OpenObservationStreamRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.open_observation_stream(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.open_observation_stream(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = InitShardsResponse; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -960,12 +957,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: InitShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.init_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.init_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = RetainShardsResponse; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -976,12 +973,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: RetainShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.retain_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.retain_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = TruncateShardsResponse; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -992,12 +989,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: TruncateShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.truncate_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.truncate_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = CloseShardsResponse; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -1008,12 +1005,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: CloseShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.close_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.close_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerIngesterServiceClient { type Response = DecommissionResponse; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -1024,15 +1021,16 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: DecommissionRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.decommission(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.decommission(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct IngesterServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerIngesterServiceClient, persist_svc: quickwit_common::tower::BoxService< PersistRequest, PersistResponse, @@ -1079,77 +1077,61 @@ struct IngesterServiceTowerServiceStack { crate::ingest::IngestV2Error, >, } -impl Clone for IngesterServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - persist_svc: self.persist_svc.clone(), - open_replication_stream_svc: self.open_replication_stream_svc.clone(), - open_fetch_stream_svc: self.open_fetch_stream_svc.clone(), - open_observation_stream_svc: self.open_observation_stream_svc.clone(), - init_shards_svc: self.init_shards_svc.clone(), - retain_shards_svc: self.retain_shards_svc.clone(), - truncate_shards_svc: self.truncate_shards_svc.clone(), - close_shards_svc: self.close_shards_svc.clone(), - decommission_svc: self.decommission_svc.clone(), - } - } -} #[async_trait::async_trait] impl IngesterService for IngesterServiceTowerServiceStack { async fn persist( - &mut self, + &self, request: PersistRequest, ) -> crate::ingest::IngestV2Result { - self.persist_svc.ready().await?.call(request).await + self.persist_svc.clone().ready().await?.call(request).await } async fn open_replication_stream( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result> { - self.open_replication_stream_svc.ready().await?.call(request).await + self.open_replication_stream_svc.clone().ready().await?.call(request).await } async fn open_fetch_stream( - &mut self, + &self, request: OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result> { - self.open_fetch_stream_svc.ready().await?.call(request).await + self.open_fetch_stream_svc.clone().ready().await?.call(request).await } async fn open_observation_stream( - &mut self, + &self, request: OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result> { - self.open_observation_stream_svc.ready().await?.call(request).await + self.open_observation_stream_svc.clone().ready().await?.call(request).await } async fn init_shards( - &mut self, + &self, request: InitShardsRequest, ) -> crate::ingest::IngestV2Result { - self.init_shards_svc.ready().await?.call(request).await + self.init_shards_svc.clone().ready().await?.call(request).await } async fn retain_shards( - &mut self, + &self, request: RetainShardsRequest, ) -> crate::ingest::IngestV2Result { - self.retain_shards_svc.ready().await?.call(request).await + self.retain_shards_svc.clone().ready().await?.call(request).await } async fn truncate_shards( - &mut self, + &self, request: TruncateShardsRequest, ) -> crate::ingest::IngestV2Result { - self.truncate_shards_svc.ready().await?.call(request).await + self.truncate_shards_svc.clone().ready().await?.call(request).await } async fn close_shards( - &mut self, + &self, request: CloseShardsRequest, ) -> crate::ingest::IngestV2Result { - self.close_shards_svc.ready().await?.call(request).await + self.close_shards_svc.clone().ready().await?.call(request).await } async fn decommission( - &mut self, + &self, request: DecommissionRequest, ) -> crate::ingest::IngestV2Result { - self.decommission_svc.ready().await?.call(request).await + self.decommission_svc.clone().ready().await?.call(request).await } } type PersistLayer = quickwit_common::tower::BoxLayer< @@ -1687,7 +1669,8 @@ impl IngesterServiceTowerLayerStack { where T: IngesterService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerIngesterServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -1695,25 +1678,25 @@ impl IngesterServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> IngesterServiceClient { - self.build_from_boxed( - Box::new( - IngesterServiceClient::from_channel(addr, channel, max_message_size), - ), - ) + let client = IngesterServiceClient::from_channel( + addr, + channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> IngesterServiceClient { - self.build_from_boxed( - Box::new( - IngesterServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = IngesterServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -1723,22 +1706,27 @@ impl IngesterServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, IngesterServiceMailbox: IngesterService, { - self.build_from_boxed(Box::new(IngesterServiceMailbox::new(mailbox))) + let inner_client = InnerIngesterServiceClient( + std::sync::Arc::new(IngesterServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: MockIngesterService) -> IngesterServiceClient { - self.build_from_boxed(Box::new(IngesterServiceClient::from_mock(mock))) + let client = IngesterServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerIngesterServiceClient, ) -> IngesterServiceClient { let persist_svc = self .persist_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let open_replication_stream_svc = self @@ -1746,7 +1734,7 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let open_fetch_stream_svc = self @@ -1754,7 +1742,7 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let open_observation_stream_svc = self @@ -1762,7 +1750,7 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let init_shards_svc = self @@ -1770,7 +1758,7 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let retain_shards_svc = self @@ -1778,7 +1766,7 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let truncate_shards_svc = self @@ -1786,7 +1774,7 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let close_shards_svc = self @@ -1794,7 +1782,7 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let decommission_svc = self @@ -1802,11 +1790,11 @@ impl IngesterServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = IngesterServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, persist_svc, open_replication_stream_svc, open_fetch_stream_svc, @@ -1957,58 +1945,58 @@ where >, { async fn persist( - &mut self, + &self, request: PersistRequest, ) -> crate::ingest::IngestV2Result { - self.call(request).await + self.clone().call(request).await } async fn open_replication_stream( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result> { - self.call(request).await + self.clone().call(request).await } async fn open_fetch_stream( - &mut self, + &self, request: OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result> { - self.call(request).await + self.clone().call(request).await } async fn open_observation_stream( - &mut self, + &self, request: OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result> { - self.call(request).await + self.clone().call(request).await } async fn init_shards( - &mut self, + &self, request: InitShardsRequest, ) -> crate::ingest::IngestV2Result { - self.call(request).await + self.clone().call(request).await } async fn retain_shards( - &mut self, + &self, request: RetainShardsRequest, ) -> crate::ingest::IngestV2Result { - self.call(request).await + self.clone().call(request).await } async fn truncate_shards( - &mut self, + &self, request: TruncateShardsRequest, ) -> crate::ingest::IngestV2Result { - self.call(request).await + self.clone().call(request).await } async fn close_shards( - &mut self, + &self, request: CloseShardsRequest, ) -> crate::ingest::IngestV2Result { - self.call(request).await + self.clone().call(request).await } async fn decommission( - &mut self, + &self, request: DecommissionRequest, ) -> crate::ingest::IngestV2Result { - self.call(request).await + self.clone().call(request).await } } #[derive(Debug, Clone)] @@ -2046,10 +2034,11 @@ where T::Future: Send, { async fn persist( - &mut self, + &self, request: PersistRequest, ) -> crate::ingest::IngestV2Result { self.inner + .clone() .persist(request) .await .map(|response| response.into_inner()) @@ -2059,10 +2048,11 @@ where )) } async fn open_replication_stream( - &mut self, + &self, request: quickwit_common::ServiceStream, ) -> crate::ingest::IngestV2Result> { self.inner + .clone() .open_replication_stream(request) .await .map(|response| { @@ -2080,10 +2070,11 @@ where )) } async fn open_fetch_stream( - &mut self, + &self, request: OpenFetchStreamRequest, ) -> crate::ingest::IngestV2Result> { self.inner + .clone() .open_fetch_stream(request) .await .map(|response| { @@ -2101,10 +2092,11 @@ where )) } async fn open_observation_stream( - &mut self, + &self, request: OpenObservationStreamRequest, ) -> crate::ingest::IngestV2Result> { self.inner + .clone() .open_observation_stream(request) .await .map(|response| { @@ -2122,10 +2114,11 @@ where )) } async fn init_shards( - &mut self, + &self, request: InitShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner + .clone() .init_shards(request) .await .map(|response| response.into_inner()) @@ -2135,10 +2128,11 @@ where )) } async fn retain_shards( - &mut self, + &self, request: RetainShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner + .clone() .retain_shards(request) .await .map(|response| response.into_inner()) @@ -2148,10 +2142,11 @@ where )) } async fn truncate_shards( - &mut self, + &self, request: TruncateShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner + .clone() .truncate_shards(request) .await .map(|response| response.into_inner()) @@ -2161,10 +2156,11 @@ where )) } async fn close_shards( - &mut self, + &self, request: CloseShardsRequest, ) -> crate::ingest::IngestV2Result { self.inner + .clone() .close_shards(request) .await .map(|response| response.into_inner()) @@ -2174,10 +2170,11 @@ where )) } async fn decommission( - &mut self, + &self, request: DecommissionRequest, ) -> crate::ingest::IngestV2Result { self.inner + .clone() .decommission(request) .await .map(|response| response.into_inner()) @@ -2189,14 +2186,16 @@ where } #[derive(Debug)] pub struct IngesterServiceGrpcServerAdapter { - inner: Box, + inner: InnerIngesterServiceClient, } impl IngesterServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: IngesterService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerIngesterServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -2207,7 +2206,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .persist(request.into_inner()) .await .map(tonic::Response::new) @@ -2221,7 +2220,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request>, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .open_replication_stream({ let streaming: tonic::Streaming<_> = request.into_inner(); quickwit_common::ServiceStream::from(streaming) @@ -2240,7 +2239,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .open_fetch_stream(request.into_inner()) .await .map(|stream| tonic::Response::new( @@ -2256,7 +2255,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .open_observation_stream(request.into_inner()) .await .map(|stream| tonic::Response::new( @@ -2269,7 +2268,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .init_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -2280,7 +2279,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .retain_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -2291,7 +2290,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .truncate_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -2302,7 +2301,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .close_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -2313,7 +2312,7 @@ for IngesterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .decommission(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs index 42abc49279b..dbc0f389539 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.ingest.router.rs @@ -125,25 +125,20 @@ impl RpcName for IngestRequestV2 { } #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait IngestRouterService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait IngestRouterService: std::fmt::Debug + Send + Sync + 'static { /// Ingests batches of documents for one or multiple indexes. /// TODO: Describe error cases and how to handle them. async fn ingest( - &mut self, + &self, request: IngestRequestV2, ) -> crate::ingest::IngestV2Result; } -dyn_clone::clone_trait_object!(IngestRouterService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockIngestRouterService { - fn clone(&self) -> Self { - MockIngestRouterService::new() - } -} #[derive(Debug, Clone)] pub struct IngestRouterServiceClient { - inner: Box, + inner: InnerIngestRouterServiceClient, } +#[derive(Debug, Clone)] +struct InnerIngestRouterServiceClient(std::sync::Arc); impl IngestRouterServiceClient { pub fn new(instance: T) -> Self where @@ -155,7 +150,9 @@ impl IngestRouterServiceClient { MockIngestRouterService > (), "`MockIngestRouterService` must be wrapped in a `MockIngestRouterServiceWrapper`: use `IngestRouterServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerIngestRouterServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -216,7 +213,7 @@ impl IngestRouterServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockIngestRouterService) -> Self { let mock_wrapper = mock_ingest_router_service::MockIngestRouterServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -228,23 +225,23 @@ impl IngestRouterServiceClient { #[async_trait::async_trait] impl IngestRouterService for IngestRouterServiceClient { async fn ingest( - &mut self, + &self, request: IngestRequestV2, ) -> crate::ingest::IngestV2Result { - self.inner.ingest(request).await + self.inner.0.ingest(request).await } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_ingest_router_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockIngestRouterServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl IngestRouterService for MockIngestRouterServiceWrapper { async fn ingest( - &mut self, + &self, request: super::IngestRequestV2, ) -> crate::ingest::IngestV2Result { self.inner.lock().await.ingest(request).await @@ -254,7 +251,7 @@ pub mod mock_ingest_router_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerIngestRouterServiceClient { type Response = IngestResponseV2; type Error = crate::ingest::IngestV2Error; type Future = BoxFuture; @@ -265,36 +262,29 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: IngestRequestV2) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.ingest(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.ingest(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct IngestRouterServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerIngestRouterServiceClient, ingest_svc: quickwit_common::tower::BoxService< IngestRequestV2, IngestResponseV2, crate::ingest::IngestV2Error, >, } -impl Clone for IngestRouterServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - ingest_svc: self.ingest_svc.clone(), - } - } -} #[async_trait::async_trait] impl IngestRouterService for IngestRouterServiceTowerServiceStack { async fn ingest( - &mut self, + &self, request: IngestRequestV2, ) -> crate::ingest::IngestV2Result { - self.ingest_svc.ready().await?.call(request).await + self.ingest_svc.clone().ready().await?.call(request).await } } type IngestLayer = quickwit_common::tower::BoxLayer< @@ -366,7 +356,8 @@ impl IngestRouterServiceTowerLayerStack { where T: IngestRouterService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerIngestRouterServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -374,25 +365,25 @@ impl IngestRouterServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> IngestRouterServiceClient { - self.build_from_boxed( - Box::new( - IngestRouterServiceClient::from_channel(addr, channel, max_message_size), - ), - ) + let client = IngestRouterServiceClient::from_channel( + addr, + channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> IngestRouterServiceClient { - self.build_from_boxed( - Box::new( - IngestRouterServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = IngestRouterServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -402,29 +393,34 @@ impl IngestRouterServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, IngestRouterServiceMailbox: IngestRouterService, { - self.build_from_boxed(Box::new(IngestRouterServiceMailbox::new(mailbox))) + let inner_client = InnerIngestRouterServiceClient( + std::sync::Arc::new(IngestRouterServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock( self, mock: MockIngestRouterService, ) -> IngestRouterServiceClient { - self.build_from_boxed(Box::new(IngestRouterServiceClient::from_mock(mock))) + let client = IngestRouterServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerIngestRouterServiceClient, ) -> IngestRouterServiceClient { let ingest_svc = self .ingest_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = IngestRouterServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, ingest_svc, }; IngestRouterServiceClient::new(tower_svc_stack) @@ -510,10 +506,10 @@ where >, { async fn ingest( - &mut self, + &self, request: IngestRequestV2, ) -> crate::ingest::IngestV2Result { - self.call(request).await + self.clone().call(request).await } } #[derive(Debug, Clone)] @@ -551,10 +547,11 @@ where T::Future: Send, { async fn ingest( - &mut self, + &self, request: IngestRequestV2, ) -> crate::ingest::IngestV2Result { self.inner + .clone() .ingest(request) .await .map(|response| response.into_inner()) @@ -566,14 +563,16 @@ where } #[derive(Debug)] pub struct IngestRouterServiceGrpcServerAdapter { - inner: Box, + inner: InnerIngestRouterServiceClient, } impl IngestRouterServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: IngestRouterService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerIngestRouterServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -584,7 +583,7 @@ for IngestRouterServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .ingest(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs index d3537e02799..4c1a3021894 100644 --- a/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs +++ b/quickwit/quickwit-proto/src/codegen/quickwit/quickwit.metastore.rs @@ -30,6 +30,8 @@ pub struct UpdateIndexRequest { pub search_settings_json: ::prost::alloc::string::String, #[prost(string, optional, tag = "3")] pub retention_policy_json: ::core::option::Option<::prost::alloc::string::String>, + #[prost(string, tag = "4")] + pub indexing_settings_json: ::prost::alloc::string::String, } #[derive(serde::Serialize, serde::Deserialize, utoipa::ToSchema)] #[allow(clippy::derive_partial_eq_without_eq)] @@ -757,108 +759,108 @@ pub type MetastoreServiceStream = quickwit_common::ServiceStream< >; #[cfg_attr(any(test, feature = "testsuite"), mockall::automock)] #[async_trait::async_trait] -pub trait MetastoreService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync + 'static { +pub trait MetastoreService: std::fmt::Debug + Send + Sync + 'static { /// Creates an index. /// /// This API creates a new index in the metastore. /// An error will occur if an index that already exists in the storage is specified. async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> crate::metastore::MetastoreResult; /// Update an index. async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> crate::metastore::MetastoreResult; /// Returns the `IndexMetadata` of an index identified by its IndexID or its IndexUID. async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> crate::metastore::MetastoreResult; /// Fetches the metadata of a list of indexes identified by their Index IDs or UIDs. async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult; /// Gets an indexes metadatas. async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult; /// Deletes an index async fn delete_index( - &mut self, + &self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult; /// Streams splits from index. async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> crate::metastore::MetastoreResult>; /// Stages several splits. async fn stage_splits( - &mut self, + &self, request: StageSplitsRequest, ) -> crate::metastore::MetastoreResult; /// Publishes split. async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> crate::metastore::MetastoreResult; /// Marks splits for deletion. async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult; /// Deletes splits. async fn delete_splits( - &mut self, + &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult; /// Adds source. async fn add_source( - &mut self, + &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult; /// Toggles source. async fn toggle_source( - &mut self, + &self, request: ToggleSourceRequest, ) -> crate::metastore::MetastoreResult; /// Removes source. async fn delete_source( - &mut self, + &self, request: DeleteSourceRequest, ) -> crate::metastore::MetastoreResult; /// Resets source checkpoint. async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult; /// Gets last opstamp for a given `index_id`. async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult; /// Creates a delete task. async fn create_delete_task( - &mut self, + &self, request: DeleteQuery, ) -> crate::metastore::MetastoreResult; /// Updates splits `delete_opstamp`. async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult; /// Lists delete tasks with `delete_task.opstamp` > `opstamp_start` for a given `index_id`. async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult; /// Lists splits with `split.delete_opstamp` < `delete_opstamp` for a given `index_id`. async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult; /// Shard API @@ -868,7 +870,7 @@ pub trait MetastoreService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync /// independently. Responses list the requests that succeeded or failed in the fields `successes` and /// `failures`. async fn open_shards( - &mut self, + &self, request: OpenShardsRequest, ) -> crate::metastore::MetastoreResult; /// Acquires a set of shards for indexing. This RPC locks the shards for publishing thanks to a publish token and only @@ -881,58 +883,53 @@ pub trait MetastoreService: std::fmt::Debug + dyn_clone::DynClone + Send + Sync /// For this reason, AcquireShards.acquire_shards may return less subresponse than there was in the request. /// Also they may be returned in any order. async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult; /// Deletes a set of shards. This RPC deletes the shards from the metastore. /// If the shard did not exist to begin with, the operation is successful and does not return any error. async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult; async fn list_shards( - &mut self, + &self, request: ListShardsRequest, ) -> crate::metastore::MetastoreResult; /// Creates an index template. async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult; /// Fetches an index template. async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult; /// Finds matching index templates. async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult; /// Returns all the index templates. async fn list_index_templates( - &mut self, + &self, request: ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult; /// Deletes index templates. async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult; - async fn check_connectivity(&mut self) -> anyhow::Result<()>; + async fn check_connectivity(&self) -> anyhow::Result<()>; fn endpoints(&self) -> Vec; } -dyn_clone::clone_trait_object!(MetastoreService); -#[cfg(any(test, feature = "testsuite"))] -impl Clone for MockMetastoreService { - fn clone(&self) -> Self { - MockMetastoreService::new() - } -} #[derive(Debug, Clone)] pub struct MetastoreServiceClient { - inner: Box, + inner: InnerMetastoreServiceClient, } +#[derive(Debug, Clone)] +struct InnerMetastoreServiceClient(std::sync::Arc); impl MetastoreServiceClient { pub fn new(instance: T) -> Self where @@ -944,7 +941,9 @@ impl MetastoreServiceClient { MockMetastoreService > (), "`MockMetastoreService` must be wrapped in a `MockMetastoreServiceWrapper`: use `MetastoreServiceClient::from_mock(mock)` to instantiate the client" ); - Self { inner: Box::new(instance) } + Self { + inner: InnerMetastoreServiceClient(std::sync::Arc::new(instance)), + } } pub fn as_grpc_service( &self, @@ -1005,7 +1004,7 @@ impl MetastoreServiceClient { #[cfg(any(test, feature = "testsuite"))] pub fn from_mock(mock: MockMetastoreService) -> Self { let mock_wrapper = mock_metastore_service::MockMetastoreServiceWrapper { - inner: std::sync::Arc::new(tokio::sync::Mutex::new(mock)), + inner: tokio::sync::Mutex::new(mock), }; Self::new(mock_wrapper) } @@ -1017,233 +1016,233 @@ impl MetastoreServiceClient { #[async_trait::async_trait] impl MetastoreService for MetastoreServiceClient { async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> crate::metastore::MetastoreResult { - self.inner.create_index(request).await + self.inner.0.create_index(request).await } async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> crate::metastore::MetastoreResult { - self.inner.update_index(request).await + self.inner.0.update_index(request).await } async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.inner.index_metadata(request).await + self.inner.0.index_metadata(request).await } async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.inner.indexes_metadata(request).await + self.inner.0.indexes_metadata(request).await } async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.inner.list_indexes_metadata(request).await + self.inner.0.list_indexes_metadata(request).await } async fn delete_index( - &mut self, + &self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult { - self.inner.delete_index(request).await + self.inner.0.delete_index(request).await } async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> crate::metastore::MetastoreResult> { - self.inner.list_splits(request).await + self.inner.0.list_splits(request).await } async fn stage_splits( - &mut self, + &self, request: StageSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.stage_splits(request).await + self.inner.0.stage_splits(request).await } async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.publish_splits(request).await + self.inner.0.publish_splits(request).await } async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { - self.inner.mark_splits_for_deletion(request).await + self.inner.0.mark_splits_for_deletion(request).await } async fn delete_splits( - &mut self, + &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.delete_splits(request).await + self.inner.0.delete_splits(request).await } async fn add_source( - &mut self, + &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult { - self.inner.add_source(request).await + self.inner.0.add_source(request).await } async fn toggle_source( - &mut self, + &self, request: ToggleSourceRequest, ) -> crate::metastore::MetastoreResult { - self.inner.toggle_source(request).await + self.inner.0.toggle_source(request).await } async fn delete_source( - &mut self, + &self, request: DeleteSourceRequest, ) -> crate::metastore::MetastoreResult { - self.inner.delete_source(request).await + self.inner.0.delete_source(request).await } async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult { - self.inner.reset_source_checkpoint(request).await + self.inner.0.reset_source_checkpoint(request).await } async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { - self.inner.last_delete_opstamp(request).await + self.inner.0.last_delete_opstamp(request).await } async fn create_delete_task( - &mut self, + &self, request: DeleteQuery, ) -> crate::metastore::MetastoreResult { - self.inner.create_delete_task(request).await + self.inner.0.create_delete_task(request).await } async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { - self.inner.update_splits_delete_opstamp(request).await + self.inner.0.update_splits_delete_opstamp(request).await } async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult { - self.inner.list_delete_tasks(request).await + self.inner.0.list_delete_tasks(request).await } async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.list_stale_splits(request).await + self.inner.0.list_stale_splits(request).await } async fn open_shards( - &mut self, + &self, request: OpenShardsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.open_shards(request).await + self.inner.0.open_shards(request).await } async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.acquire_shards(request).await + self.inner.0.acquire_shards(request).await } async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.delete_shards(request).await + self.inner.0.delete_shards(request).await } async fn list_shards( - &mut self, + &self, request: ListShardsRequest, ) -> crate::metastore::MetastoreResult { - self.inner.list_shards(request).await + self.inner.0.list_shards(request).await } async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { - self.inner.create_index_template(request).await + self.inner.0.create_index_template(request).await } async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { - self.inner.get_index_template(request).await + self.inner.0.get_index_template(request).await } async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult { - self.inner.find_index_template_matches(request).await + self.inner.0.find_index_template_matches(request).await } async fn list_index_templates( - &mut self, + &self, request: ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { - self.inner.list_index_templates(request).await + self.inner.0.list_index_templates(request).await } async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { - self.inner.delete_index_templates(request).await + self.inner.0.delete_index_templates(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { - self.inner.check_connectivity().await + async fn check_connectivity(&self) -> anyhow::Result<()> { + self.inner.0.check_connectivity().await } fn endpoints(&self) -> Vec { - self.inner.endpoints() + self.inner.0.endpoints() } } #[cfg(any(test, feature = "testsuite"))] pub mod mock_metastore_service { use super::*; - #[derive(Debug, Clone)] + #[derive(Debug)] pub struct MockMetastoreServiceWrapper { - pub(super) inner: std::sync::Arc>, + pub(super) inner: tokio::sync::Mutex, } #[async_trait::async_trait] impl MetastoreService for MockMetastoreServiceWrapper { async fn create_index( - &mut self, + &self, request: super::CreateIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.create_index(request).await } async fn update_index( - &mut self, + &self, request: super::UpdateIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.update_index(request).await } async fn index_metadata( - &mut self, + &self, request: super::IndexMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.index_metadata(request).await } async fn indexes_metadata( - &mut self, + &self, request: super::IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.indexes_metadata(request).await } async fn list_indexes_metadata( - &mut self, + &self, request: super::ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.list_indexes_metadata(request).await } async fn delete_index( - &mut self, + &self, request: super::DeleteIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_index(request).await } async fn list_splits( - &mut self, + &self, request: super::ListSplitsRequest, ) -> crate::metastore::MetastoreResult< MetastoreServiceStream, @@ -1251,67 +1250,67 @@ pub mod mock_metastore_service { self.inner.lock().await.list_splits(request).await } async fn stage_splits( - &mut self, + &self, request: super::StageSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.stage_splits(request).await } async fn publish_splits( - &mut self, + &self, request: super::PublishSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.publish_splits(request).await } async fn mark_splits_for_deletion( - &mut self, + &self, request: super::MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.mark_splits_for_deletion(request).await } async fn delete_splits( - &mut self, + &self, request: super::DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_splits(request).await } async fn add_source( - &mut self, + &self, request: super::AddSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.add_source(request).await } async fn toggle_source( - &mut self, + &self, request: super::ToggleSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.toggle_source(request).await } async fn delete_source( - &mut self, + &self, request: super::DeleteSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_source(request).await } async fn reset_source_checkpoint( - &mut self, + &self, request: super::ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.reset_source_checkpoint(request).await } async fn last_delete_opstamp( - &mut self, + &self, request: super::LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.last_delete_opstamp(request).await } async fn create_delete_task( - &mut self, + &self, request: super::DeleteQuery, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.create_delete_task(request).await } async fn update_splits_delete_opstamp( - &mut self, + &self, request: super::UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult< super::UpdateSplitsDeleteOpstampResponse, @@ -1319,72 +1318,72 @@ pub mod mock_metastore_service { self.inner.lock().await.update_splits_delete_opstamp(request).await } async fn list_delete_tasks( - &mut self, + &self, request: super::ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.list_delete_tasks(request).await } async fn list_stale_splits( - &mut self, + &self, request: super::ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.list_stale_splits(request).await } async fn open_shards( - &mut self, + &self, request: super::OpenShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.open_shards(request).await } async fn acquire_shards( - &mut self, + &self, request: super::AcquireShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.acquire_shards(request).await } async fn delete_shards( - &mut self, + &self, request: super::DeleteShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_shards(request).await } async fn list_shards( - &mut self, + &self, request: super::ListShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.list_shards(request).await } async fn create_index_template( - &mut self, + &self, request: super::CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.create_index_template(request).await } async fn get_index_template( - &mut self, + &self, request: super::GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.get_index_template(request).await } async fn find_index_template_matches( - &mut self, + &self, request: super::FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.find_index_template_matches(request).await } async fn list_index_templates( - &mut self, + &self, request: super::ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.list_index_templates(request).await } async fn delete_index_templates( - &mut self, + &self, request: super::DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { self.inner.lock().await.delete_index_templates(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { self.inner.lock().await.check_connectivity().await } fn endpoints(&self) -> Vec { @@ -1395,7 +1394,7 @@ pub mod mock_metastore_service { pub type BoxFuture = std::pin::Pin< Box> + Send + 'static>, >; -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = CreateIndexResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1406,12 +1405,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: CreateIndexRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.create_index(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.create_index(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = IndexMetadataResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1422,12 +1421,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: UpdateIndexRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.update_index(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.update_index(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = IndexMetadataResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1438,12 +1437,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: IndexMetadataRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.index_metadata(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.index_metadata(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = IndexesMetadataResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1454,12 +1453,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: IndexesMetadataRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.indexes_metadata(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.indexes_metadata(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = ListIndexesMetadataResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1470,12 +1469,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ListIndexesMetadataRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.list_indexes_metadata(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.list_indexes_metadata(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1486,12 +1485,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: DeleteIndexRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_index(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.delete_index(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = MetastoreServiceStream; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1502,12 +1501,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ListSplitsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.list_splits(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.list_splits(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1518,12 +1517,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: StageSplitsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.stage_splits(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.stage_splits(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1534,12 +1533,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: PublishSplitsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.publish_splits(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.publish_splits(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1550,12 +1549,12 @@ impl tower::Service for Box std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: MarkSplitsForDeletionRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.mark_splits_for_deletion(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.mark_splits_for_deletion(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1566,12 +1565,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: DeleteSplitsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_splits(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.delete_splits(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1582,12 +1581,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: AddSourceRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.add_source(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.add_source(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1598,12 +1597,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ToggleSourceRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.toggle_source(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.toggle_source(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1614,12 +1613,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: DeleteSourceRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_source(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.delete_source(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1630,12 +1629,12 @@ impl tower::Service for Box std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ResetSourceCheckpointRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.reset_source_checkpoint(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.reset_source_checkpoint(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = LastDeleteOpstampResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1646,12 +1645,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: LastDeleteOpstampRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.last_delete_opstamp(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.last_delete_opstamp(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = DeleteTask; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1662,12 +1661,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: DeleteQuery) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.create_delete_task(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.create_delete_task(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = UpdateSplitsDeleteOpstampResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1678,12 +1677,12 @@ impl tower::Service for Box Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.update_splits_delete_opstamp(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.update_splits_delete_opstamp(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = ListDeleteTasksResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1694,12 +1693,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ListDeleteTasksRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.list_delete_tasks(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.list_delete_tasks(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = ListSplitsResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1710,12 +1709,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ListStaleSplitsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.list_stale_splits(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.list_stale_splits(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = OpenShardsResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1726,12 +1725,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: OpenShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.open_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.open_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = AcquireShardsResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1742,12 +1741,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: AcquireShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.acquire_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.acquire_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = DeleteShardsResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1758,12 +1757,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: DeleteShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.delete_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = ListShardsResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1774,12 +1773,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ListShardsRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.list_shards(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.list_shards(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1790,12 +1789,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: CreateIndexTemplateRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.create_index_template(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.create_index_template(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = GetIndexTemplateResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1806,12 +1805,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: GetIndexTemplateRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.get_index_template(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.get_index_template(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = FindIndexTemplateMatchesResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1822,12 +1821,12 @@ impl tower::Service for Box Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.find_index_template_matches(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.find_index_template_matches(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = ListIndexTemplatesResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1838,12 +1837,12 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: ListIndexTemplatesRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.list_index_templates(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.list_index_templates(request).await }; Box::pin(fut) } } -impl tower::Service for Box { +impl tower::Service for InnerMetastoreServiceClient { type Response = EmptyResponse; type Error = crate::metastore::MetastoreError; type Future = BoxFuture; @@ -1854,15 +1853,16 @@ impl tower::Service for Box { std::task::Poll::Ready(Ok(())) } fn call(&mut self, request: DeleteIndexTemplatesRequest) -> Self::Future { - let mut svc = self.clone(); - let fut = async move { svc.delete_index_templates(request).await }; + let svc = self.clone(); + let fut = async move { svc.0.delete_index_templates(request).await }; Box::pin(fut) } } /// A tower service stack is a set of tower services. #[derive(Debug)] struct MetastoreServiceTowerServiceStack { - inner: Box, + #[allow(dead_code)] + inner: InnerMetastoreServiceClient, create_index_svc: quickwit_common::tower::BoxService< CreateIndexRequest, CreateIndexResponse, @@ -2009,227 +2009,187 @@ struct MetastoreServiceTowerServiceStack { crate::metastore::MetastoreError, >, } -impl Clone for MetastoreServiceTowerServiceStack { - fn clone(&self) -> Self { - Self { - inner: self.inner.clone(), - create_index_svc: self.create_index_svc.clone(), - update_index_svc: self.update_index_svc.clone(), - index_metadata_svc: self.index_metadata_svc.clone(), - indexes_metadata_svc: self.indexes_metadata_svc.clone(), - list_indexes_metadata_svc: self.list_indexes_metadata_svc.clone(), - delete_index_svc: self.delete_index_svc.clone(), - list_splits_svc: self.list_splits_svc.clone(), - stage_splits_svc: self.stage_splits_svc.clone(), - publish_splits_svc: self.publish_splits_svc.clone(), - mark_splits_for_deletion_svc: self.mark_splits_for_deletion_svc.clone(), - delete_splits_svc: self.delete_splits_svc.clone(), - add_source_svc: self.add_source_svc.clone(), - toggle_source_svc: self.toggle_source_svc.clone(), - delete_source_svc: self.delete_source_svc.clone(), - reset_source_checkpoint_svc: self.reset_source_checkpoint_svc.clone(), - last_delete_opstamp_svc: self.last_delete_opstamp_svc.clone(), - create_delete_task_svc: self.create_delete_task_svc.clone(), - update_splits_delete_opstamp_svc: self - .update_splits_delete_opstamp_svc - .clone(), - list_delete_tasks_svc: self.list_delete_tasks_svc.clone(), - list_stale_splits_svc: self.list_stale_splits_svc.clone(), - open_shards_svc: self.open_shards_svc.clone(), - acquire_shards_svc: self.acquire_shards_svc.clone(), - delete_shards_svc: self.delete_shards_svc.clone(), - list_shards_svc: self.list_shards_svc.clone(), - create_index_template_svc: self.create_index_template_svc.clone(), - get_index_template_svc: self.get_index_template_svc.clone(), - find_index_template_matches_svc: self - .find_index_template_matches_svc - .clone(), - list_index_templates_svc: self.list_index_templates_svc.clone(), - delete_index_templates_svc: self.delete_index_templates_svc.clone(), - } - } -} #[async_trait::async_trait] impl MetastoreService for MetastoreServiceTowerServiceStack { async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> crate::metastore::MetastoreResult { - self.create_index_svc.ready().await?.call(request).await + self.create_index_svc.clone().ready().await?.call(request).await } async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> crate::metastore::MetastoreResult { - self.update_index_svc.ready().await?.call(request).await + self.update_index_svc.clone().ready().await?.call(request).await } async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.index_metadata_svc.ready().await?.call(request).await + self.index_metadata_svc.clone().ready().await?.call(request).await } async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.indexes_metadata_svc.ready().await?.call(request).await + self.indexes_metadata_svc.clone().ready().await?.call(request).await } async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.list_indexes_metadata_svc.ready().await?.call(request).await + self.list_indexes_metadata_svc.clone().ready().await?.call(request).await } async fn delete_index( - &mut self, + &self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult { - self.delete_index_svc.ready().await?.call(request).await + self.delete_index_svc.clone().ready().await?.call(request).await } async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> crate::metastore::MetastoreResult> { - self.list_splits_svc.ready().await?.call(request).await + self.list_splits_svc.clone().ready().await?.call(request).await } async fn stage_splits( - &mut self, + &self, request: StageSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.stage_splits_svc.ready().await?.call(request).await + self.stage_splits_svc.clone().ready().await?.call(request).await } async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.publish_splits_svc.ready().await?.call(request).await + self.publish_splits_svc.clone().ready().await?.call(request).await } async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { - self.mark_splits_for_deletion_svc.ready().await?.call(request).await + self.mark_splits_for_deletion_svc.clone().ready().await?.call(request).await } async fn delete_splits( - &mut self, + &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.delete_splits_svc.ready().await?.call(request).await + self.delete_splits_svc.clone().ready().await?.call(request).await } async fn add_source( - &mut self, + &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult { - self.add_source_svc.ready().await?.call(request).await + self.add_source_svc.clone().ready().await?.call(request).await } async fn toggle_source( - &mut self, + &self, request: ToggleSourceRequest, ) -> crate::metastore::MetastoreResult { - self.toggle_source_svc.ready().await?.call(request).await + self.toggle_source_svc.clone().ready().await?.call(request).await } async fn delete_source( - &mut self, + &self, request: DeleteSourceRequest, ) -> crate::metastore::MetastoreResult { - self.delete_source_svc.ready().await?.call(request).await + self.delete_source_svc.clone().ready().await?.call(request).await } async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult { - self.reset_source_checkpoint_svc.ready().await?.call(request).await + self.reset_source_checkpoint_svc.clone().ready().await?.call(request).await } async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { - self.last_delete_opstamp_svc.ready().await?.call(request).await + self.last_delete_opstamp_svc.clone().ready().await?.call(request).await } async fn create_delete_task( - &mut self, + &self, request: DeleteQuery, ) -> crate::metastore::MetastoreResult { - self.create_delete_task_svc.ready().await?.call(request).await + self.create_delete_task_svc.clone().ready().await?.call(request).await } async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { - self.update_splits_delete_opstamp_svc.ready().await?.call(request).await + self.update_splits_delete_opstamp_svc.clone().ready().await?.call(request).await } async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult { - self.list_delete_tasks_svc.ready().await?.call(request).await + self.list_delete_tasks_svc.clone().ready().await?.call(request).await } async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.list_stale_splits_svc.ready().await?.call(request).await + self.list_stale_splits_svc.clone().ready().await?.call(request).await } async fn open_shards( - &mut self, + &self, request: OpenShardsRequest, ) -> crate::metastore::MetastoreResult { - self.open_shards_svc.ready().await?.call(request).await + self.open_shards_svc.clone().ready().await?.call(request).await } async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult { - self.acquire_shards_svc.ready().await?.call(request).await + self.acquire_shards_svc.clone().ready().await?.call(request).await } async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult { - self.delete_shards_svc.ready().await?.call(request).await + self.delete_shards_svc.clone().ready().await?.call(request).await } async fn list_shards( - &mut self, + &self, request: ListShardsRequest, ) -> crate::metastore::MetastoreResult { - self.list_shards_svc.ready().await?.call(request).await + self.list_shards_svc.clone().ready().await?.call(request).await } async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { - self.create_index_template_svc.ready().await?.call(request).await + self.create_index_template_svc.clone().ready().await?.call(request).await } async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { - self.get_index_template_svc.ready().await?.call(request).await + self.get_index_template_svc.clone().ready().await?.call(request).await } async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult { - self.find_index_template_matches_svc.ready().await?.call(request).await + self.find_index_template_matches_svc.clone().ready().await?.call(request).await } async fn list_index_templates( - &mut self, + &self, request: ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { - self.list_index_templates_svc.ready().await?.call(request).await + self.list_index_templates_svc.clone().ready().await?.call(request).await } async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { - self.delete_index_templates_svc.ready().await?.call(request).await + self.delete_index_templates_svc.clone().ready().await?.call(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { - self.inner.check_connectivity().await + async fn check_connectivity(&self) -> anyhow::Result<()> { + self.inner.0.check_connectivity().await } fn endpoints(&self) -> Vec { - self.inner.endpoints() + self.inner.0.endpoints() } } type CreateIndexLayer = quickwit_common::tower::BoxLayer< @@ -3941,7 +3901,8 @@ impl MetastoreServiceTowerLayerStack { where T: MetastoreService, { - self.build_from_boxed(Box::new(instance)) + let inner_client = InnerMetastoreServiceClient(std::sync::Arc::new(instance)); + self.build_from_inner_client(inner_client) } pub fn build_from_channel( self, @@ -3949,25 +3910,25 @@ impl MetastoreServiceTowerLayerStack { channel: tonic::transport::Channel, max_message_size: bytesize::ByteSize, ) -> MetastoreServiceClient { - self.build_from_boxed( - Box::new( - MetastoreServiceClient::from_channel(addr, channel, max_message_size), - ), - ) + let client = MetastoreServiceClient::from_channel( + addr, + channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_balance_channel( self, balance_channel: quickwit_common::tower::BalanceChannel, max_message_size: bytesize::ByteSize, ) -> MetastoreServiceClient { - self.build_from_boxed( - Box::new( - MetastoreServiceClient::from_balance_channel( - balance_channel, - max_message_size, - ), - ), - ) + let client = MetastoreServiceClient::from_balance_channel( + balance_channel, + max_message_size, + ); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } pub fn build_from_mailbox( self, @@ -3977,22 +3938,27 @@ impl MetastoreServiceTowerLayerStack { A: quickwit_actors::Actor + std::fmt::Debug + Send + 'static, MetastoreServiceMailbox: MetastoreService, { - self.build_from_boxed(Box::new(MetastoreServiceMailbox::new(mailbox))) + let inner_client = InnerMetastoreServiceClient( + std::sync::Arc::new(MetastoreServiceMailbox::new(mailbox)), + ); + self.build_from_inner_client(inner_client) } #[cfg(any(test, feature = "testsuite"))] pub fn build_from_mock(self, mock: MockMetastoreService) -> MetastoreServiceClient { - self.build_from_boxed(Box::new(MetastoreServiceClient::from_mock(mock))) + let client = MetastoreServiceClient::from_mock(mock); + let inner_client = client.inner; + self.build_from_inner_client(inner_client) } - fn build_from_boxed( + fn build_from_inner_client( self, - boxed_instance: Box, + inner_client: InnerMetastoreServiceClient, ) -> MetastoreServiceClient { let create_index_svc = self .create_index_layers .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let update_index_svc = self @@ -4000,7 +3966,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let index_metadata_svc = self @@ -4008,7 +3974,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let indexes_metadata_svc = self @@ -4016,7 +3982,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let list_indexes_metadata_svc = self @@ -4024,7 +3990,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let delete_index_svc = self @@ -4032,7 +3998,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let list_splits_svc = self @@ -4040,7 +4006,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let stage_splits_svc = self @@ -4048,7 +4014,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let publish_splits_svc = self @@ -4056,7 +4022,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let mark_splits_for_deletion_svc = self @@ -4064,7 +4030,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let delete_splits_svc = self @@ -4072,7 +4038,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let add_source_svc = self @@ -4080,7 +4046,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let toggle_source_svc = self @@ -4088,7 +4054,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let delete_source_svc = self @@ -4096,7 +4062,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let reset_source_checkpoint_svc = self @@ -4104,7 +4070,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let last_delete_opstamp_svc = self @@ -4112,7 +4078,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let create_delete_task_svc = self @@ -4120,7 +4086,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let update_splits_delete_opstamp_svc = self @@ -4128,7 +4094,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let list_delete_tasks_svc = self @@ -4136,7 +4102,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let list_stale_splits_svc = self @@ -4144,7 +4110,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let open_shards_svc = self @@ -4152,7 +4118,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let acquire_shards_svc = self @@ -4160,7 +4126,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let delete_shards_svc = self @@ -4168,7 +4134,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let list_shards_svc = self @@ -4176,7 +4142,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let create_index_template_svc = self @@ -4184,7 +4150,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let get_index_template_svc = self @@ -4192,7 +4158,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let find_index_template_matches_svc = self @@ -4200,7 +4166,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let list_index_templates_svc = self @@ -4208,7 +4174,7 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let delete_index_templates_svc = self @@ -4216,11 +4182,11 @@ impl MetastoreServiceTowerLayerStack { .into_iter() .rev() .fold( - quickwit_common::tower::BoxService::new(boxed_instance.clone()), + quickwit_common::tower::BoxService::new(inner_client.clone()), |svc, layer| layer.layer(svc), ); let tower_svc_stack = MetastoreServiceTowerServiceStack { - inner: boxed_instance.clone(), + inner: inner_client, create_index_svc, update_index_svc, index_metadata_svc, @@ -4523,180 +4489,180 @@ where >, { async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn delete_index( - &mut self, + &self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> crate::metastore::MetastoreResult> { - self.call(request).await + self.clone().call(request).await } async fn stage_splits( - &mut self, + &self, request: StageSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn delete_splits( - &mut self, + &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn add_source( - &mut self, + &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn toggle_source( - &mut self, + &self, request: ToggleSourceRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn delete_source( - &mut self, + &self, request: DeleteSourceRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn create_delete_task( - &mut self, + &self, request: DeleteQuery, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn open_shards( - &mut self, + &self, request: OpenShardsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn list_shards( - &mut self, + &self, request: ListShardsRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn list_index_templates( - &mut self, + &self, request: ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { - self.call(request).await + self.clone().call(request).await } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { if self.inner.is_disconnected() { anyhow::bail!("actor `{}` is disconnected", self.inner.actor_instance_id()) } @@ -4744,10 +4710,11 @@ where T::Future: Send, { async fn create_index( - &mut self, + &self, request: CreateIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .create_index(request) .await .map(|response| response.into_inner()) @@ -4757,10 +4724,11 @@ where )) } async fn update_index( - &mut self, + &self, request: UpdateIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .update_index(request) .await .map(|response| response.into_inner()) @@ -4770,10 +4738,11 @@ where )) } async fn index_metadata( - &mut self, + &self, request: IndexMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .index_metadata(request) .await .map(|response| response.into_inner()) @@ -4783,10 +4752,11 @@ where )) } async fn indexes_metadata( - &mut self, + &self, request: IndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .indexes_metadata(request) .await .map(|response| response.into_inner()) @@ -4796,10 +4766,11 @@ where )) } async fn list_indexes_metadata( - &mut self, + &self, request: ListIndexesMetadataRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .list_indexes_metadata(request) .await .map(|response| response.into_inner()) @@ -4809,10 +4780,11 @@ where )) } async fn delete_index( - &mut self, + &self, request: DeleteIndexRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .delete_index(request) .await .map(|response| response.into_inner()) @@ -4822,10 +4794,11 @@ where )) } async fn list_splits( - &mut self, + &self, request: ListSplitsRequest, ) -> crate::metastore::MetastoreResult> { self.inner + .clone() .list_splits(request) .await .map(|response| { @@ -4843,10 +4816,11 @@ where )) } async fn stage_splits( - &mut self, + &self, request: StageSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .stage_splits(request) .await .map(|response| response.into_inner()) @@ -4856,10 +4830,11 @@ where )) } async fn publish_splits( - &mut self, + &self, request: PublishSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .publish_splits(request) .await .map(|response| response.into_inner()) @@ -4869,10 +4844,11 @@ where )) } async fn mark_splits_for_deletion( - &mut self, + &self, request: MarkSplitsForDeletionRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .mark_splits_for_deletion(request) .await .map(|response| response.into_inner()) @@ -4882,10 +4858,11 @@ where )) } async fn delete_splits( - &mut self, + &self, request: DeleteSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .delete_splits(request) .await .map(|response| response.into_inner()) @@ -4895,10 +4872,11 @@ where )) } async fn add_source( - &mut self, + &self, request: AddSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .add_source(request) .await .map(|response| response.into_inner()) @@ -4908,10 +4886,11 @@ where )) } async fn toggle_source( - &mut self, + &self, request: ToggleSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .toggle_source(request) .await .map(|response| response.into_inner()) @@ -4921,10 +4900,11 @@ where )) } async fn delete_source( - &mut self, + &self, request: DeleteSourceRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .delete_source(request) .await .map(|response| response.into_inner()) @@ -4934,10 +4914,11 @@ where )) } async fn reset_source_checkpoint( - &mut self, + &self, request: ResetSourceCheckpointRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .reset_source_checkpoint(request) .await .map(|response| response.into_inner()) @@ -4947,10 +4928,11 @@ where )) } async fn last_delete_opstamp( - &mut self, + &self, request: LastDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .last_delete_opstamp(request) .await .map(|response| response.into_inner()) @@ -4960,10 +4942,11 @@ where )) } async fn create_delete_task( - &mut self, + &self, request: DeleteQuery, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .create_delete_task(request) .await .map(|response| response.into_inner()) @@ -4973,10 +4956,11 @@ where )) } async fn update_splits_delete_opstamp( - &mut self, + &self, request: UpdateSplitsDeleteOpstampRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .update_splits_delete_opstamp(request) .await .map(|response| response.into_inner()) @@ -4986,10 +4970,11 @@ where )) } async fn list_delete_tasks( - &mut self, + &self, request: ListDeleteTasksRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .list_delete_tasks(request) .await .map(|response| response.into_inner()) @@ -4999,10 +4984,11 @@ where )) } async fn list_stale_splits( - &mut self, + &self, request: ListStaleSplitsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .list_stale_splits(request) .await .map(|response| response.into_inner()) @@ -5012,10 +4998,11 @@ where )) } async fn open_shards( - &mut self, + &self, request: OpenShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .open_shards(request) .await .map(|response| response.into_inner()) @@ -5025,10 +5012,11 @@ where )) } async fn acquire_shards( - &mut self, + &self, request: AcquireShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .acquire_shards(request) .await .map(|response| response.into_inner()) @@ -5038,10 +5026,11 @@ where )) } async fn delete_shards( - &mut self, + &self, request: DeleteShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .delete_shards(request) .await .map(|response| response.into_inner()) @@ -5051,10 +5040,11 @@ where )) } async fn list_shards( - &mut self, + &self, request: ListShardsRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .list_shards(request) .await .map(|response| response.into_inner()) @@ -5064,10 +5054,11 @@ where )) } async fn create_index_template( - &mut self, + &self, request: CreateIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .create_index_template(request) .await .map(|response| response.into_inner()) @@ -5077,10 +5068,11 @@ where )) } async fn get_index_template( - &mut self, + &self, request: GetIndexTemplateRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .get_index_template(request) .await .map(|response| response.into_inner()) @@ -5090,10 +5082,11 @@ where )) } async fn find_index_template_matches( - &mut self, + &self, request: FindIndexTemplateMatchesRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .find_index_template_matches(request) .await .map(|response| response.into_inner()) @@ -5103,10 +5096,11 @@ where )) } async fn list_index_templates( - &mut self, + &self, request: ListIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .list_index_templates(request) .await .map(|response| response.into_inner()) @@ -5116,10 +5110,11 @@ where )) } async fn delete_index_templates( - &mut self, + &self, request: DeleteIndexTemplatesRequest, ) -> crate::metastore::MetastoreResult { self.inner + .clone() .delete_index_templates(request) .await .map(|response| response.into_inner()) @@ -5128,7 +5123,7 @@ where DeleteIndexTemplatesRequest::rpc_name(), )) } - async fn check_connectivity(&mut self) -> anyhow::Result<()> { + async fn check_connectivity(&self) -> anyhow::Result<()> { if self.connection_addrs_rx.borrow().len() == 0 { anyhow::bail!("no server currently available") } @@ -5146,14 +5141,16 @@ where } #[derive(Debug)] pub struct MetastoreServiceGrpcServerAdapter { - inner: Box, + inner: InnerMetastoreServiceClient, } impl MetastoreServiceGrpcServerAdapter { pub fn new(instance: T) -> Self where T: MetastoreService, { - Self { inner: Box::new(instance) } + Self { + inner: InnerMetastoreServiceClient(std::sync::Arc::new(instance)), + } } } #[async_trait::async_trait] @@ -5164,7 +5161,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .create_index(request.into_inner()) .await .map(tonic::Response::new) @@ -5175,7 +5172,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .update_index(request.into_inner()) .await .map(tonic::Response::new) @@ -5186,7 +5183,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .index_metadata(request.into_inner()) .await .map(tonic::Response::new) @@ -5197,7 +5194,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .indexes_metadata(request.into_inner()) .await .map(tonic::Response::new) @@ -5208,7 +5205,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .list_indexes_metadata(request.into_inner()) .await .map(tonic::Response::new) @@ -5219,7 +5216,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .delete_index(request.into_inner()) .await .map(tonic::Response::new) @@ -5233,7 +5230,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .list_splits(request.into_inner()) .await .map(|stream| tonic::Response::new( @@ -5246,7 +5243,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .stage_splits(request.into_inner()) .await .map(tonic::Response::new) @@ -5257,7 +5254,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .publish_splits(request.into_inner()) .await .map(tonic::Response::new) @@ -5268,7 +5265,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .mark_splits_for_deletion(request.into_inner()) .await .map(tonic::Response::new) @@ -5279,7 +5276,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .delete_splits(request.into_inner()) .await .map(tonic::Response::new) @@ -5290,7 +5287,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .add_source(request.into_inner()) .await .map(tonic::Response::new) @@ -5301,7 +5298,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .toggle_source(request.into_inner()) .await .map(tonic::Response::new) @@ -5312,7 +5309,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .delete_source(request.into_inner()) .await .map(tonic::Response::new) @@ -5323,7 +5320,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .reset_source_checkpoint(request.into_inner()) .await .map(tonic::Response::new) @@ -5334,7 +5331,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .last_delete_opstamp(request.into_inner()) .await .map(tonic::Response::new) @@ -5345,7 +5342,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .create_delete_task(request.into_inner()) .await .map(tonic::Response::new) @@ -5356,7 +5353,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .update_splits_delete_opstamp(request.into_inner()) .await .map(tonic::Response::new) @@ -5367,7 +5364,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .list_delete_tasks(request.into_inner()) .await .map(tonic::Response::new) @@ -5378,7 +5375,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .list_stale_splits(request.into_inner()) .await .map(tonic::Response::new) @@ -5389,7 +5386,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .open_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -5400,7 +5397,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .acquire_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -5411,7 +5408,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .delete_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -5422,7 +5419,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .list_shards(request.into_inner()) .await .map(tonic::Response::new) @@ -5433,7 +5430,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .create_index_template(request.into_inner()) .await .map(tonic::Response::new) @@ -5444,7 +5441,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .get_index_template(request.into_inner()) .await .map(tonic::Response::new) @@ -5455,7 +5452,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .find_index_template_matches(request.into_inner()) .await .map(tonic::Response::new) @@ -5466,7 +5463,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .list_index_templates(request.into_inner()) .await .map(tonic::Response::new) @@ -5477,7 +5474,7 @@ for MetastoreServiceGrpcServerAdapter { request: tonic::Request, ) -> Result, tonic::Status> { self.inner - .clone() + .0 .delete_index_templates(request.into_inner()) .await .map(tonic::Response::new) diff --git a/quickwit/quickwit-proto/src/lib.rs b/quickwit/quickwit-proto/src/lib.rs index fb432a7df27..0458e993c5b 100644 --- a/quickwit/quickwit-proto/src/lib.rs +++ b/quickwit/quickwit-proto/src/lib.rs @@ -43,8 +43,7 @@ pub mod search; pub mod types; pub use error::{GrpcServiceError, ServiceError, ServiceErrorCode}; - -use crate::search::ReportSplitsRequest; +use search::ReportSplitsRequest; pub mod jaeger { pub mod api_v2 { diff --git a/quickwit/quickwit-rest-client/src/rest_client.rs b/quickwit/quickwit-rest-client/src/rest_client.rs index fe2f796a38c..062b5389f92 100644 --- a/quickwit/quickwit-rest-client/src/rest_client.rs +++ b/quickwit/quickwit-rest-client/src/rest_client.rs @@ -26,9 +26,7 @@ use quickwit_indexing::actors::IndexingServiceCounters; pub use quickwit_ingest::CommitType; use quickwit_metastore::{IndexMetadata, Split, SplitInfo}; use quickwit_search::SearchResponseRest; -use quickwit_serve::{ - IndexUpdates, ListSplitsQueryParams, ListSplitsResponse, SearchRequestQueryString, -}; +use quickwit_serve::{ListSplitsQueryParams, ListSplitsResponse, SearchRequestQueryString}; use reqwest::header::{HeaderMap, HeaderValue, CONTENT_TYPE}; use reqwest::{Client, ClientBuilder, Method, StatusCode, Url}; use serde::Serialize; @@ -333,12 +331,12 @@ impl<'a> IndexClient<'a> { pub async fn create( &self, - index_config: impl ToString, + index_config: impl AsRef<[u8]>, config_format: ConfigFormat, overwrite: bool, ) -> Result { let header_map = header_from_config_format(config_format); - let body = Bytes::from(index_config.to_string()); + let body = Bytes::copy_from_slice(index_config.as_ref()); let response = self .transport .send( @@ -357,13 +355,22 @@ impl<'a> IndexClient<'a> { pub async fn update( &self, index_id: &str, - index_updates: IndexUpdates, + index_config: impl AsRef<[u8]>, + config_format: ConfigFormat, ) -> Result { - let body = Bytes::from(serde_json::to_string(&index_updates)?); + let header_map = header_from_config_format(config_format); + let body = Bytes::copy_from_slice(index_config.as_ref()); let path = format!("indexes/{index_id}"); let response = self .transport - .send::<()>(Method::PUT, &path, None, None, Some(body), self.timeout) + .send::<()>( + Method::PUT, + &path, + Some(header_map), + None, + Some(body), + self.timeout, + ) .await?; let index_metadata = response.deserialize().await?; Ok(index_metadata) @@ -490,11 +497,11 @@ impl<'a> SourceClient<'a> { pub async fn create( &self, - source_config_input: impl ToString, + source_config_input: impl AsRef<[u8]>, config_format: ConfigFormat, ) -> Result { let header_map = header_from_config_format(config_format); - let source_config_bytes: Bytes = Bytes::from(source_config_input.to_string()); + let source_config_bytes = Bytes::copy_from_slice(source_config_input.as_ref()); let response = self .transport .send::<()>( diff --git a/quickwit/quickwit-serve/src/delete_task_api/handler.rs b/quickwit/quickwit-serve/src/delete_task_api/handler.rs index 755549d0c08..d00d882c141 100644 --- a/quickwit/quickwit-serve/src/delete_task_api/handler.rs +++ b/quickwit/quickwit-serve/src/delete_task_api/handler.rs @@ -98,7 +98,7 @@ pub fn get_delete_tasks_handler( // `get_delete_tasks_handler` and consequently we get the mailbox in `get_delete_tasks` signature. pub async fn get_delete_tasks( index_id: IndexId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult> { let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_uid: IndexUid = metastore @@ -145,7 +145,7 @@ pub fn post_delete_tasks_handler( pub async fn post_delete_request( index_id: IndexId, delete_request: DeleteQueryRequest, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> Result { let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let metadata = metastore diff --git a/quickwit/quickwit-serve/src/developer_api/debug.rs b/quickwit/quickwit-serve/src/developer_api/debug.rs index a967d7c22aa..3a1c32fbbc3 100644 --- a/quickwit/quickwit-serve/src/developer_api/debug.rs +++ b/quickwit/quickwit-serve/src/developer_api/debug.rs @@ -110,7 +110,7 @@ async fn get_node_debug_infos( for ready_node in ready_nodes { if node_id_patterns.matches(ready_node.node_id()) { let node_id = ready_node.node_id().to_owned(); - let mut client = DeveloperServiceClient::from_channel( + let client = DeveloperServiceClient::from_channel( ready_node.grpc_advertise_addr(), ready_node.channel(), DeveloperApiServer::MAX_GRPC_MESSAGE_SIZE, diff --git a/quickwit/quickwit-serve/src/developer_api/server.rs b/quickwit/quickwit-serve/src/developer_api/server.rs index 03791b1b4f9..bf4ad37b7fd 100644 --- a/quickwit/quickwit-serve/src/developer_api/server.rs +++ b/quickwit/quickwit-serve/src/developer_api/server.rs @@ -69,7 +69,7 @@ impl DeveloperApiServer { #[async_trait] impl DeveloperService for DeveloperApiServer { async fn get_debug_info( - &mut self, + &self, request: GetDebugInfoRequest, ) -> DeveloperResult { let roles: HashSet = request @@ -144,7 +144,7 @@ mod tests { let node_config = Arc::new(NodeConfig::for_test()); - let mut developer_api_server = DeveloperApiServer { + let developer_api_server = DeveloperApiServer { node_config, cluster, control_plane_mailbox_opt: None, diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/bulk.rs b/quickwit/quickwit-serve/src/elasticsearch_api/bulk.rs index 24df36ac3e1..5bd5ab43829 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/bulk.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/bulk.rs @@ -82,7 +82,7 @@ async fn elastic_ingest_bulk( default_index_id: Option, body: Body, bulk_options: ElasticBulkOptions, - mut ingest_service: IngestServiceClient, + ingest_service: IngestServiceClient, ingest_router: IngestRouterServiceClient, ) -> Result { if enable_ingest_v2() || bulk_options.enable_ingest_v2 { diff --git a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs index c0c7ecf735a..c1aa0c19018 100644 --- a/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs +++ b/quickwit/quickwit-serve/src/elasticsearch_api/bulk_v2.rs @@ -77,7 +77,7 @@ pub(crate) async fn elastic_bulk_ingest_v2( default_index_id: Option, body: Body, bulk_options: ElasticBulkOptions, - mut ingest_router: IngestRouterServiceClient, + ingest_router: IngestRouterServiceClient, ) -> Result { let now = Instant::now(); let mut ingest_request_builder = IngestRequestV2Builder::default(); diff --git a/quickwit/quickwit-serve/src/index_api/mod.rs b/quickwit/quickwit-serve/src/index_api/mod.rs index 64d58cddb94..a11878ac264 100644 --- a/quickwit/quickwit-serve/src/index_api/mod.rs +++ b/quickwit/quickwit-serve/src/index_api/mod.rs @@ -20,6 +20,6 @@ mod rest_handler; pub use self::rest_handler::{ - get_index_metadata_handler, index_management_handlers, IndexApi, IndexUpdates, - ListSplitsQueryParams, ListSplitsResponse, + get_index_metadata_handler, index_management_handlers, IndexApi, ListSplitsQueryParams, + ListSplitsResponse, }; diff --git a/quickwit/quickwit-serve/src/index_api/rest_handler.rs b/quickwit/quickwit-serve/src/index_api/rest_handler.rs index 12e2f22b367..f51a83c1f0c 100644 --- a/quickwit/quickwit-serve/src/index_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/index_api/rest_handler.rs @@ -22,9 +22,8 @@ use std::sync::Arc; use bytes::Bytes; use quickwit_common::uri::Uri; use quickwit_config::{ - load_source_config_from_user_config, validate_index_id_pattern, ConfigFormat, NodeConfig, - RetentionPolicy, SearchSettings, SourceConfig, SourceParams, CLI_SOURCE_ID, - INGEST_API_SOURCE_ID, + load_index_config_update, load_source_config_from_user_config, validate_index_id_pattern, + ConfigFormat, NodeConfig, SourceConfig, SourceParams, CLI_SOURCE_ID, INGEST_API_SOURCE_ID, }; use quickwit_doc_mapper::{analyze_text, TokenizerConfig}; use quickwit_index_management::{IndexService, IndexServiceError}; @@ -68,7 +67,7 @@ use crate::with_arg; toggle_source, delete_source, ), - components(schemas(ToggleSource, SplitsForDeletion, IndexStats, IndexUpdates)) + components(schemas(ToggleSource, SplitsForDeletion, IndexStats)) )] pub struct IndexApi; @@ -129,7 +128,7 @@ pub fn get_index_metadata_handler( async fn get_index_metadata( index_id: IndexId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult { info!(index_id = %index_id, "get-index-metadata"); let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); @@ -194,7 +193,7 @@ struct IndexStats { /// Describes an index. async fn describe_index( index_id: IndexId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult { let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_metadata = metastore @@ -322,7 +321,7 @@ pub struct ListSplitsResponse { async fn list_splits( index_id: IndexId, list_split_query: ListSplitsQueryParams, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult { let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_uid: IndexUid = metastore @@ -399,7 +398,7 @@ struct SplitsForDeletion { async fn mark_splits_for_deletion( index_id: IndexId, splits_for_deletion: SplitsForDeletion, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult<()> { let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_uid: IndexUid = metastore @@ -449,7 +448,7 @@ fn mark_splits_for_deletion_handler( /// Gets indexes metadata. async fn list_indexes_metadata( list_indexes_params: ListIndexesQueryParams, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult> { let list_indexes_metata_request = if let Some(index_id_patterns) = list_indexes_params.index_id_patterns { @@ -529,22 +528,14 @@ async fn create_index( .await } -/// The body of the index update request. All fields will be replaced in the -/// existing configuration. -#[derive(Deserialize, Serialize, Debug, Eq, PartialEq, Default, utoipa::ToSchema)] -#[serde(deny_unknown_fields)] // Remove when adding new fields to allow to ensure forward compatibility -pub struct IndexUpdates { - pub search_settings: SearchSettings, - #[serde(rename = "retention_policy")] - pub retention_policy_opt: Option, -} - fn update_index_handler( metastore: MetastoreServiceClient, ) -> impl Filter + Clone { warp::path!("indexes" / String) .and(warp::put()) - .and(json_body()) + .and(extract_config_format()) + .and(warp::body::content_length_limit(1024 * 1024)) + .and(warp::filters::body::bytes()) .and(with_arg(metastore)) .then(update_index) .map(log_failure("failed to update index")) @@ -556,9 +547,9 @@ fn update_index_handler( put, tag = "Indexes", path = "/indexes/{index_id}", - request_body = IndexUpdates, + request_body = VersionedIndexConfig, responses( - (status = 200, description = "Successfully updated the index configuration.") + (status = 200, description = "Successfully updated the index configuration.", body = VersionedIndexMetadata) ), params( ("index_id" = String, Path, description = "The index ID to update."), @@ -566,26 +557,37 @@ fn update_index_handler( )] /// Updates an existing index. /// -/// This endpoint has PUT semantics, which means that all the updatable fields of the index -/// configuration are replaced by the values specified in the request. In particular, omitting an -/// optional field like `retention_policy` will delete the associated configuration. +/// This endpoint follows PUT semantics, which means that all the fields of the +/// current configuration are replaced by the values specified in this request +/// or the associated defaults. In particular, if the field is optional (e.g. +/// `retention_policy`), omitting it will delete the associated configuration. +/// If the new configuration file contains updates that cannot be applied, the +/// request fails, and none of the updates are applied. async fn update_index( - index_id: IndexId, - request: IndexUpdates, - mut metastore: MetastoreServiceClient, + target_index_id: IndexId, + config_format: ConfigFormat, + index_config_bytes: Bytes, + metastore: MetastoreServiceClient, ) -> Result { - info!(index_id = %index_id, "update-index"); - let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); - let index_uid: IndexUid = metastore + info!(index_id = %target_index_id, "update-index"); + + let index_metadata_request = IndexMetadataRequest::for_index_id(target_index_id.to_string()); + let current_index_metadata = metastore .index_metadata(index_metadata_request) .await? - .deserialize_index_metadata()? - .index_uid; + .deserialize_index_metadata()?; + let index_uid = current_index_metadata.index_uid.clone(); + let current_index_config = current_index_metadata.into_index_config(); + + let new_index_config = + load_index_config_update(config_format, &index_config_bytes, ¤t_index_config) + .map_err(IndexServiceError::InvalidConfig)?; let update_request = UpdateIndexRequest::try_from_updates( index_uid, - &request.search_settings, - &request.retention_policy_opt, + &new_index_config.search_settings, + &new_index_config.retention_policy_opt, + &new_index_config.indexing_settings, )?; let update_resp = metastore.update_index(update_request).await?; Ok(update_resp.deserialize_index_metadata()?) @@ -737,7 +739,7 @@ fn get_source_handler( async fn get_source( index_id: IndexId, source_id: SourceId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult { info!(index_id = %index_id, source_id = %source_id, "get-source"); let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); @@ -783,7 +785,7 @@ fn reset_source_checkpoint_handler( async fn reset_source_checkpoint( index_id: IndexId, source_id: SourceId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult<()> { let index_metadata_resquest = IndexMetadataRequest::for_index_id(index_id.to_string()); let index_uid: IndexUid = metastore @@ -838,7 +840,7 @@ async fn toggle_source( index_id: IndexId, source_id: SourceId, toggle_source: ToggleSource, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> Result<(), IndexServiceError> { info!(index_id = %index_id, source_id = %source_id, enable = toggle_source.enable, "toggle-source"); let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); @@ -889,7 +891,7 @@ fn delete_source_handler( async fn delete_source( index_id: IndexId, source_id: SourceId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> Result<(), IndexServiceError> { info!(index_id = %index_id, source_id = %source_id, "delete-source"); let index_metadata_request = IndexMetadataRequest::for_index_id(index_id.to_string()); @@ -1517,7 +1519,7 @@ mod tests { #[tokio::test] async fn test_create_delete_index_and_source() { - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_service = IndexService::new(metastore.clone(), StorageResolver::unconfigured()); let mut node_config = NodeConfig::for_test(); node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); @@ -1783,7 +1785,7 @@ mod tests { #[tokio::test] async fn test_update_index() { - let mut metastore = metastore_for_test(); + let metastore = metastore_for_test(); let index_service = IndexService::new(metastore.clone(), StorageResolver::unconfigured()); let mut node_config = NodeConfig::for_test(); node_config.default_index_root_uri = Uri::for_test("file:///default-index-root-uri"); @@ -1813,7 +1815,7 @@ mod tests { .path("/indexes/hdfs-logs") .method("PUT") .json(&true) - .body(r#"{"search_settings":{"default_search_fields":["severity_text","body"]}}"#) + .body(r#"{"version": "0.7", "index_id": "hdfs-logs", "doc_mapping": {"field_mappings":[{"name": "timestamp", "type": "i64", "fast": true, "indexed": true}]},"search_settings":{"default_search_fields":["severity_text", "body"]}}"#) .reply(&index_management_handler) .await; assert_eq!(resp.status(), 200); diff --git a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs index df89b5f0e4d..afd0bfb752b 100644 --- a/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/ingest_api/rest_handler.rs @@ -119,7 +119,7 @@ async fn ingest_v2( index_id: IndexId, body: Body, ingest_options: IngestOptions, - mut ingest_router: IngestRouterServiceClient, + ingest_router: IngestRouterServiceClient, ) -> Result { let mut doc_batch_builder = DocBatchV2Builder::default(); @@ -206,7 +206,7 @@ async fn ingest( index_id: IndexId, body: Body, ingest_options: IngestOptions, - mut ingest_service: IngestServiceClient, + ingest_service: IngestServiceClient, ) -> Result { if disable_ingest_v1() { let message = "ingest v1 is disabled: environment variable `QW_DISABLE_INGEST_V1` is set"; @@ -254,7 +254,7 @@ fn tail_filter() -> impl Filter + Clone /// Returns the last few ingested documents. async fn tail_endpoint( index_id: IndexId, - mut ingest_service: IngestServiceClient, + ingest_service: IngestServiceClient, ) -> Result { let fetch_response = ingest_service.tail(TailRequest { index_id }).await?; Ok(fetch_response) diff --git a/quickwit/quickwit-serve/src/lib.rs b/quickwit/quickwit-serve/src/lib.rs index 141dea1c13d..df57f67a585 100644 --- a/quickwit/quickwit-serve/src/lib.rs +++ b/quickwit/quickwit-serve/src/lib.rs @@ -120,7 +120,7 @@ use tracing::{debug, error, info, warn}; use warp::{Filter, Rejection}; pub use crate::build_info::{BuildInfo, RuntimeInfo}; -pub use crate::index_api::{IndexUpdates, ListSplitsQueryParams, ListSplitsResponse}; +pub use crate::index_api::{ListSplitsQueryParams, ListSplitsResponse}; pub use crate::metrics::SERVE_METRICS; use crate::rate_modulator::RateModulator; #[cfg(test)] @@ -1151,7 +1151,7 @@ fn with_arg(arg: T) -> impl Filter, grpc_readiness_signal_rx: oneshot::Receiver<()>, rest_readiness_signal_rx: oneshot::Receiver<()>, @@ -1199,7 +1199,7 @@ async fn node_readiness_reporting_task( async fn check_cluster_configuration( services: &HashSet, peer_seeds: &[String], - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> anyhow::Result<()> { if !services.contains(&QuickwitService::Metastore) || peer_seeds.is_empty() { return Ok(()); diff --git a/quickwit/quickwit-serve/src/template_api/rest_handler.rs b/quickwit/quickwit-serve/src/template_api/rest_handler.rs index 8b4c76cba56..a9fc9bb0de0 100644 --- a/quickwit/quickwit-serve/src/template_api/rest_handler.rs +++ b/quickwit/quickwit-serve/src/template_api/rest_handler.rs @@ -85,7 +85,7 @@ fn create_index_template_handler( async fn create_index_template( body: Bytes, config_format: ConfigFormat, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult { let index_template: IndexTemplate = config_format @@ -132,7 +132,7 @@ fn get_index_template_handler( /// Retrieves the index template identified by `template_id`. async fn get_index_template( template_id: IndexTemplateId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult { let get_index_template_request = GetIndexTemplateRequest { template_id }; let get_index_template_response = metastore @@ -170,7 +170,7 @@ async fn update_index_template( template_id: IndexTemplateId, body: Bytes, config_format: ConfigFormat, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult { let mut json_value: JsonValue = config_format @@ -223,7 +223,7 @@ fn delete_index_template_handler( /// Deletes the index template identified by the provided `template_id`. async fn delete_index_template( template_id: IndexTemplateId, - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult<()> { let template_ids = vec![template_id]; let delete_index_templates_request = DeleteIndexTemplatesRequest { template_ids }; @@ -254,7 +254,7 @@ fn list_index_templates_handler( )] /// Retrieves all the index templates stored in the metastore. async fn list_index_templates( - mut metastore: MetastoreServiceClient, + metastore: MetastoreServiceClient, ) -> MetastoreResult> { let list_index_templates_request = ListIndexTemplatesRequest {}; let list_index_templates_response = metastore