Skip to content

[Pull-based ingestion] Support multi-threaded writes in pull based ingestion #17912

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),

## [Unreleased 3.x]
### Added
- Add multi-threaded writer support in pull-based ingestion ([#17912](https://github.com/opensearch-project/OpenSearch/pull/17912))

### Changed

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -200,4 +200,31 @@ public void testUpdateWithoutIDField() throws Exception {
return 30 == (Integer) response.getHits().getHits()[0].getSourceAsMap().get("age");
});
}

public void testMultiThreadedWrites() throws Exception {
// create index with 5 writer threads
createIndexWithDefaultSettings(indexName, 1, 0, 5);
ensureGreen(indexName);

// Step 1: Produce messages
for (int i = 0; i < 1000; i++) {
produceData(Integer.toString(i), "name" + i, "25");
}

waitForState(() -> {
SearchResponse searchableDocsResponse = client().prepareSearch(indexName).setSize(2000).setPreference("_only_local").get();
return searchableDocsResponse.getHits().getTotalHits().value() == 1000;
});

// Step 2: Produce an update message and validate
for (int i = 0; i < 1000; i++) {
produceData(Integer.toString(i), "name" + i, "30");
}

waitForState(() -> {
RangeQueryBuilder query = new RangeQueryBuilder("age").gte(28);
SearchResponse response = client().prepareSearch(indexName).setQuery(query).get();
return response.getHits().getTotalHits().value() == 1000;
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -177,10 +177,10 @@ protected ResumeIngestionResponse resumeIngestion(String indexName) throws Execu
}

protected void createIndexWithDefaultSettings(int numShards, int numReplicas) {
createIndexWithDefaultSettings(indexName, numShards, numReplicas);
createIndexWithDefaultSettings(indexName, numShards, numReplicas, 1);
}

protected void createIndexWithDefaultSettings(String indexName, int numShards, int numReplicas) {
protected void createIndexWithDefaultSettings(String indexName, int numShards, int numReplicas, int numProcessorThreads) {
createIndex(
indexName,
Settings.builder()
Expand All @@ -191,6 +191,7 @@ protected void createIndexWithDefaultSettings(String indexName, int numShards, i
.put("ingestion_source.param.topic", topicName)
.put("ingestion_source.param.bootstrap_servers", kafka.getBootstrapServers())
.put("index.replication.type", "SEGMENT")
.put("ingestion_source.num_processor_threads", numProcessorThreads)
// set custom kafka consumer properties
.put("ingestion_source.param.fetch.min.bytes", 30000)
.put("ingestion_source.param.enable.auto.commit", false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,7 @@ public void testErrorStrategy() throws Exception {
.setSettings(Settings.builder().put("ingestion_source.error_strategy", "drop"))
.get();
waitForState(() -> "drop".equalsIgnoreCase(getSettings(indexName, "index.ingestion_source.error_strategy")));
resumeIngestion(indexName);
waitForSearchableDocs(2, Arrays.asList(node));
}

Expand Down Expand Up @@ -248,8 +249,8 @@ public void testPaginatedGetIngestionState() throws ExecutionException, Interrup
internalCluster().startClusterManagerOnlyNode();
internalCluster().startDataOnlyNode();
internalCluster().startDataOnlyNode();
createIndexWithDefaultSettings("index1", 5, 0);
createIndexWithDefaultSettings("index2", 5, 0);
createIndexWithDefaultSettings("index1", 5, 0, 1);
createIndexWithDefaultSettings("index2", 5, 0, 1);
ensureGreen("index1");
ensureGreen("index2");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -831,6 +831,18 @@ public Iterator<Setting<?>> settings() {
Property.Dynamic
);

/**
* Defines the number of processor threads that will write to the lucene index.
*/
public static final String SETTING_INGESTION_SOURCE_NUM_PROCESSOR_THREADS = "index.ingestion_source.num_processor_threads";
public static final Setting<Integer> INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING = Setting.intSetting(
SETTING_INGESTION_SOURCE_NUM_PROCESSOR_THREADS,
1,
1,
Setting.Property.IndexScope,
Setting.Property.Final
);

public static final Setting.AffixSetting<Object> INGESTION_SOURCE_PARAMS_SETTING = Setting.prefixKeySetting(
"index.ingestion_source.param.",
key -> new Setting<>(key, "", (value) -> {
Expand Down Expand Up @@ -1073,11 +1085,14 @@ public IngestionSource getIngestionSource() {
final Map<String, Object> ingestionSourceParams = INGESTION_SOURCE_PARAMS_SETTING.getAsMap(settings);
final long maxPollSize = INGESTION_SOURCE_MAX_POLL_SIZE.get(settings);
final int pollTimeout = INGESTION_SOURCE_POLL_TIMEOUT.get(settings);
final int numProcessorThreads = INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING.get(settings);

return new IngestionSource.Builder(ingestionSourceType).setParams(ingestionSourceParams)
.setPointerInitReset(pointerInitReset)
.setErrorStrategy(errorStrategy)
.setMaxPollSize(maxPollSize)
.setPollTimeout(pollTimeout)
.setNumProcessorThreads(numProcessorThreads)
.build();
}
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import java.util.Objects;

import static org.opensearch.cluster.metadata.IndexMetadata.INGESTION_SOURCE_MAX_POLL_SIZE;
import static org.opensearch.cluster.metadata.IndexMetadata.INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING;
import static org.opensearch.cluster.metadata.IndexMetadata.INGESTION_SOURCE_POLL_TIMEOUT;

/**
Expand All @@ -31,21 +32,24 @@ public class IngestionSource {
private final Map<String, Object> params;
private final long maxPollSize;
private final int pollTimeout;
private int numProcessorThreads;

private IngestionSource(
String type,
PointerInitReset pointerInitReset,
IngestionErrorStrategy.ErrorStrategy errorStrategy,
Map<String, Object> params,
long maxPollSize,
int pollTimeout
int pollTimeout,
int numProcessorThreads
) {
this.type = type;
this.pointerInitReset = pointerInitReset;
this.params = params;
this.errorStrategy = errorStrategy;
this.maxPollSize = maxPollSize;
this.pollTimeout = pollTimeout;
this.numProcessorThreads = numProcessorThreads;
}

public String getType() {
Expand All @@ -72,6 +76,10 @@ public int getPollTimeout() {
return pollTimeout;
}

public int getNumProcessorThreads() {
return numProcessorThreads;
}

@Override
public boolean equals(Object o) {
if (this == o) return true;
Expand All @@ -82,12 +90,13 @@ public boolean equals(Object o) {
&& Objects.equals(errorStrategy, ingestionSource.errorStrategy)
&& Objects.equals(params, ingestionSource.params)
&& Objects.equals(maxPollSize, ingestionSource.maxPollSize)
&& Objects.equals(pollTimeout, ingestionSource.pollTimeout);
&& Objects.equals(pollTimeout, ingestionSource.pollTimeout)
&& Objects.equals(numProcessorThreads, ingestionSource.numProcessorThreads);
}

@Override
public int hashCode() {
return Objects.hash(type, pointerInitReset, params, errorStrategy, maxPollSize, pollTimeout);
return Objects.hash(type, pointerInitReset, params, errorStrategy, maxPollSize, pollTimeout, numProcessorThreads);
}

@Override
Expand All @@ -108,6 +117,8 @@ public String toString() {
+ maxPollSize
+ ", pollTimeout="
+ pollTimeout
+ ", numProcessorThreads="
+ numProcessorThreads
+ '}';
}

Expand Down Expand Up @@ -163,6 +174,7 @@ public static class Builder {
private Map<String, Object> params;
private long maxPollSize = INGESTION_SOURCE_MAX_POLL_SIZE.getDefault(Settings.EMPTY);
private int pollTimeout = INGESTION_SOURCE_POLL_TIMEOUT.getDefault(Settings.EMPTY);
private int numProcessorThreads = INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING.getDefault(Settings.EMPTY);

public Builder(String type) {
this.type = type;
Expand Down Expand Up @@ -206,8 +218,13 @@ public Builder setPollTimeout(int pollTimeout) {
return this;
}

public Builder setNumProcessorThreads(int numProcessorThreads) {
this.numProcessorThreads = numProcessorThreads;
return this;
}

public IngestionSource build() {
return new IngestionSource(type, pointerInitReset, errorStrategy, params, maxPollSize, pollTimeout);
return new IngestionSource(type, pointerInitReset, errorStrategy, params, maxPollSize, pollTimeout, numProcessorThreads);
}

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -269,6 +269,9 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
IndexMetadata.INGESTION_SOURCE_POINTER_INIT_RESET_VALUE_SETTING,
IndexMetadata.INGESTION_SOURCE_PARAMS_SETTING,
IndexMetadata.INGESTION_SOURCE_ERROR_STRATEGY_SETTING,
IndexMetadata.INGESTION_SOURCE_MAX_POLL_SIZE,
IndexMetadata.INGESTION_SOURCE_POLL_TIMEOUT,
IndexMetadata.INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING,

// Settings for search replica
IndexMetadata.INDEX_NUMBER_OF_SEARCH_REPLICAS_SETTING,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,8 @@ public void start() {
ingestionErrorStrategy,
initialPollerState,
ingestionSource.getMaxPollSize(),
ingestionSource.getPollTimeout()
ingestionSource.getPollTimeout(),
ingestionSource.getNumProcessorThreads()
);
streamPoller.start();
}
Expand Down
Loading
Loading