Skip to content

Commit 0868894

Browse files
authored
Deprecate max pipeline per indexer (#4754)
* Changed the default of `default_merge_concurrency` to `2 * num_cpus / 3` We have a observed at least one case (rather extreme however) where the default settings did not make it possible for merging to keep up with indexing. * Added documentation for indexer's cpu_capacity Closes #4716 * This PR deprecates the max_num_pipeline_per_index in Kafka's source config. It also renames desired_num_pipelines into num_pipelines. Closes #4624
1 parent fc8ac18 commit 0868894

File tree

32 files changed

+198
-247
lines changed

32 files changed

+198
-247
lines changed

config/tutorials/gh-archive/kafka-source.yaml

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,8 +1,7 @@
1-
version: 0.7
1+
version: 0.8
22
source_id: kafka-source
33
source_type: kafka
4-
max_num_pipelines_per_indexer: 1
5-
desired_num_pipelines: 2
4+
num_pipelines: 2
65
params:
76
topic: gh-archive
87
client_params:

docs/configuration/source-config.md

Lines changed: 7 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -80,11 +80,10 @@ Short max poll interval durations may cause a source to crash when back pressure
8080

8181
```bash
8282
cat << EOF > source-config.yaml
83-
version: 0.7
83+
version: 0.8
8484
source_id: my-kafka-source
8585
source_type: kafka
86-
max_num_pipelines_per_indexer: 1
87-
desired_num_pipelines: 2
86+
num_pipelines: 2
8887
params:
8988
topic: my-topic
9089
client_params:
@@ -164,38 +163,13 @@ EOF
164163
./quickwit source create --index my-index --source-config source-config.yaml
165164
```
166165

167-
## Maximum number of pipelines per indexer
168-
169-
The `max_num_pipelines_per_indexer` parameter is only available for sources that can be distributed: Kafka, GCP PubSub and Pulsar(coming soon).
170-
171-
The maximum number of indexing pipelines defines the limit of pipelines spawned for the source on a given indexer.
172-
This maximum can be reached only if there are enough `desired_num_pipelines` to run.
173-
174-
:::note
175-
176-
With the following parameters, only one pipeline will run on one indexer.
177-
178-
- `max_num_pipelines_per_indexer=2`
179-
- `desired_num_pipelines=1`
180-
181-
:::
182-
183-
## Desired number of pipelines
184-
185-
`desired_num_pipelines` parameter is only available for sources that can be distributed: Kafka, GCP PubSub and Pulsar (coming soon).
186-
187-
The desired number of indexing pipelines defines the number of pipelines to run on a cluster for the source. It is a "desired"
188-
number as it cannot be reach it there is not enough indexers in
189-
the cluster.
190-
191-
:::note
192-
193-
With the following parameters, only one pipeline will start on the sole indexer.
166+
## Number of pipelines
194167

195-
- `max_num_pipelines_per_indexer=1`
196-
- `desired_num_pipelines=2`
168+
`num_pipelines` parameter is only available for sources that can be distributed: Kafka, GCP PubSub and Pulsar (coming soon).
197169

198-
:::
170+
It defines the number of pipelines to run on a cluster for the source. The actual placement of these pipelines on the different indexer
171+
will be decided by the control plane. Note that distributions of a source like Kafka is done by assigning a set of partitions to different pipelines.
172+
As a result, it is recommended to make sure the number of partitions is a multiple of the number of `num_pipelines`.
199173

200174
## Transform parameters
201175

docs/ingest-data/kafka.md

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -101,11 +101,10 @@ This tutorial assumes that the Kafka cluster is available locally on the default
101101
#
102102
# Kafka source config file.
103103
#
104-
version: 0.7
104+
version: 0.8
105105
source_id: kafka-source
106106
source_type: kafka
107-
max_num_pipelines_per_indexer: 1
108-
desired_num_pipelines: 2
107+
num_pipelines: 2
109108
params:
110109
topic: gh-archive
111110
client_params:

quickwit/quickwit-cli/src/source.rs

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -741,8 +741,7 @@ mod tests {
741741
.collect();
742742
let sources = vec![SourceConfig {
743743
source_id: "foo-source".to_string(),
744-
desired_num_pipelines: NonZeroUsize::new(1).unwrap(),
745-
max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(),
744+
num_pipelines: NonZeroUsize::new(1).unwrap(),
746745
enabled: true,
747746
source_params: SourceParams::file("path/to/file"),
748747
transform_config: None,
@@ -802,17 +801,15 @@ mod tests {
802801
let sources = [
803802
SourceConfig {
804803
source_id: "foo-source".to_string(),
805-
desired_num_pipelines: NonZeroUsize::new(1).unwrap(),
806-
max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(),
804+
num_pipelines: NonZeroUsize::new(1).unwrap(),
807805
enabled: true,
808806
source_params: SourceParams::stdin(),
809807
transform_config: None,
810808
input_format: SourceInputFormat::Json,
811809
},
812810
SourceConfig {
813811
source_id: "bar-source".to_string(),
814-
desired_num_pipelines: NonZeroUsize::new(1).unwrap(),
815-
max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(),
812+
num_pipelines: NonZeroUsize::new(1).unwrap(),
816813
enabled: true,
817814
source_params: SourceParams::stdin(),
818815
transform_config: None,

quickwit/quickwit-cli/src/tool.rs

Lines changed: 2 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -422,8 +422,7 @@ pub async fn local_ingest_docs_cli(args: LocalIngestDocsArgs) -> anyhow::Result<
422422
.map(|vrl_script| TransformConfig::new(vrl_script, None));
423423
let source_config = SourceConfig {
424424
source_id: CLI_SOURCE_ID.to_string(),
425-
max_num_pipelines_per_indexer: NonZeroUsize::new(1).expect("1 is always non-zero."),
426-
desired_num_pipelines: NonZeroUsize::new(1).expect("1 is always non-zero."),
425+
num_pipelines: NonZeroUsize::new(1).expect("1 is always non-zero."),
427426
enabled: true,
428427
source_params,
429428
transform_config,
@@ -608,8 +607,7 @@ pub async fn merge_cli(args: MergeArgs) -> anyhow::Result<()> {
608607
index_id: args.index_id,
609608
source_config: SourceConfig {
610609
source_id: args.source_id,
611-
max_num_pipelines_per_indexer: NonZeroUsize::new(1).unwrap(),
612-
desired_num_pipelines: NonZeroUsize::new(1).unwrap(),
610+
num_pipelines: NonZeroUsize::new(1).unwrap(),
613611
enabled: true,
614612
source_params: SourceParams::Vec(VecSourceParams::default()),
615613
transform_config: None,

quickwit/quickwit-config/resources/tests/source_config/kafka-source.json

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2,7 +2,6 @@
22
"version": "0.7",
33
"source_id": "hdfs-logs-kafka-source",
44
"desired_num_pipelines": 2,
5-
"max_num_pipelines_per_indexer": 2,
65
"source_type": "kafka",
76
"params": {
87
"topic": "cloudera-cluster-logs",

quickwit/quickwit-config/src/lib.rs

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ pub use crate::node_config::{
7272
enable_ingest_v2, IndexerConfig, IngestApiConfig, JaegerConfig, NodeConfig, SearcherConfig,
7373
SplitCacheLimits, DEFAULT_QW_CONFIG_PATH,
7474
};
75-
use crate::source_config::serialize::{SourceConfigV0_7, VersionedSourceConfig};
75+
use crate::source_config::serialize::{SourceConfigV0_7, SourceConfigV0_8, VersionedSourceConfig};
7676
pub use crate::storage_config::{
7777
AzureStorageConfig, FileStorageConfig, GoogleCloudStorageConfig, RamStorageConfig,
7878
S3StorageConfig, StorageBackend, StorageBackendFlavor, StorageConfig, StorageConfigs,
@@ -88,6 +88,7 @@ pub use crate::storage_config::{
8888
DocMapping,
8989
VersionedSourceConfig,
9090
SourceConfigV0_7,
91+
SourceConfigV0_8,
9192
VersionedIndexConfig,
9293
IndexConfigV0_7,
9394
VersionedIndexTemplate,

0 commit comments

Comments
 (0)