Skip to content

Commit c697e54

Browse files
Support multi-threaded writes in pull based ingestion
Signed-off-by: Varun Bharadwaj <[email protected]>
1 parent 6ff44d9 commit c697e54

File tree

17 files changed

+564
-127
lines changed

17 files changed

+564
-127
lines changed

Diff for: CHANGELOG.md

+1
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
3838
- [Star Tree] [Search] Add query changes to support unsigned-long in star tree ([#17275](https://github.com/opensearch-project/OpenSearch/pull/17275))
3939
- Add `ApproximateMatchAllQuery` that targets match_all queries and approximates sorts ([#17772](https://github.com/opensearch-project/OpenSearch/pull/17772))
4040
- Add TermsQuery support to Search GRPC endpoint ([#17888](https://github.com/opensearch-project/OpenSearch/pull/17888))
41+
- Add multi-threaded writer support in pull-based ingestion ([#17912](https://github.com/opensearch-project/OpenSearch/pull/17912))
4142

4243
### Changed
4344
- Migrate BC libs to their FIPS counterparts ([#14912](https://github.com/opensearch-project/OpenSearch/pull/14912))

Diff for: plugins/ingestion-kafka/src/internalClusterTest/java/org/opensearch/plugin/kafka/IngestFromKafkaIT.java

+27
Original file line numberDiff line numberDiff line change
@@ -200,4 +200,31 @@ public void testUpdateWithoutIDField() throws Exception {
200200
return 30 == (Integer) response.getHits().getHits()[0].getSourceAsMap().get("age");
201201
});
202202
}
203+
204+
public void testMultiThreadedWrites() throws Exception {
205+
// create index with 5 writer threads
206+
createIndexWithDefaultSettings(indexName, 1, 0, 5);
207+
ensureGreen(indexName);
208+
209+
// Step 1: Produce messages
210+
for (int i = 0; i < 1000; i++) {
211+
produceData(Integer.toString(i), "name" + i, "25");
212+
}
213+
214+
waitForState(() -> {
215+
SearchResponse searchableDocsResponse = client().prepareSearch(indexName).setSize(2000).setPreference("_only_local").get();
216+
return searchableDocsResponse.getHits().getTotalHits().value() == 1000;
217+
});
218+
219+
// Step 2: Produce an update message and validate
220+
for (int i = 0; i < 1000; i++) {
221+
produceData(Integer.toString(i), "name" + i, "30");
222+
}
223+
224+
waitForState(() -> {
225+
RangeQueryBuilder query = new RangeQueryBuilder("age").gte(28);
226+
SearchResponse response = client().prepareSearch(indexName).setQuery(query).get();
227+
return response.getHits().getTotalHits().value() == 1000;
228+
});
229+
}
203230
}

Diff for: plugins/ingestion-kafka/src/internalClusterTest/java/org/opensearch/plugin/kafka/KafkaIngestionBaseIT.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -164,10 +164,10 @@ protected ResumeIngestionResponse resumeIngestion(String indexName) throws Execu
164164
}
165165

166166
protected void createIndexWithDefaultSettings(int numShards, int numReplicas) {
167-
createIndexWithDefaultSettings(indexName, numShards, numReplicas);
167+
createIndexWithDefaultSettings(indexName, numShards, numReplicas, 1);
168168
}
169169

170-
protected void createIndexWithDefaultSettings(String indexName, int numShards, int numReplicas) {
170+
protected void createIndexWithDefaultSettings(String indexName, int numShards, int numReplicas, int numProcessorThreads) {
171171
createIndex(
172172
indexName,
173173
Settings.builder()
@@ -178,6 +178,7 @@ protected void createIndexWithDefaultSettings(String indexName, int numShards, i
178178
.put("ingestion_source.param.topic", topicName)
179179
.put("ingestion_source.param.bootstrap_servers", kafka.getBootstrapServers())
180180
.put("index.replication.type", "SEGMENT")
181+
.put("ingestion_source.num_processor_threads", numProcessorThreads)
181182
// set custom kafka consumer properties
182183
.put("ingestion_source.param.fetch.min.bytes", 30000)
183184
.put("ingestion_source.param.enable.auto.commit", false)

Diff for: plugins/ingestion-kafka/src/internalClusterTest/java/org/opensearch/plugin/kafka/RemoteStoreKafkaIT.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -246,8 +246,8 @@ public void testPaginatedGetIngestionState() throws ExecutionException, Interrup
246246
internalCluster().startClusterManagerOnlyNode();
247247
internalCluster().startDataOnlyNode();
248248
internalCluster().startDataOnlyNode();
249-
createIndexWithDefaultSettings("index1", 5, 0);
250-
createIndexWithDefaultSettings("index2", 5, 0);
249+
createIndexWithDefaultSettings("index1", 5, 0, 1);
250+
createIndexWithDefaultSettings("index2", 5, 0, 1);
251251
ensureGreen("index1");
252252
ensureGreen("index2");
253253

Diff for: server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java

+15
Original file line numberDiff line numberDiff line change
@@ -831,6 +831,18 @@ public Iterator<Setting<?>> settings() {
831831
Property.Dynamic
832832
);
833833

834+
/**
835+
* Defines the number of processor threads that will write to the lucene index.
836+
*/
837+
public static final String SETTING_INGESTION_SOURCE_NUM_PROCESSOR_THREADS = "index.ingestion_source.num_processor_threads";
838+
public static final Setting<Integer> INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING = Setting.intSetting(
839+
SETTING_INGESTION_SOURCE_NUM_PROCESSOR_THREADS,
840+
1,
841+
1,
842+
Setting.Property.IndexScope,
843+
Setting.Property.Final
844+
);
845+
834846
public static final Setting.AffixSetting<Object> INGESTION_SOURCE_PARAMS_SETTING = Setting.prefixKeySetting(
835847
"index.ingestion_source.param.",
836848
key -> new Setting<>(key, "", (value) -> {
@@ -1073,11 +1085,14 @@ public IngestionSource getIngestionSource() {
10731085
final Map<String, Object> ingestionSourceParams = INGESTION_SOURCE_PARAMS_SETTING.getAsMap(settings);
10741086
final long maxPollSize = INGESTION_SOURCE_MAX_POLL_SIZE.get(settings);
10751087
final int pollTimeout = INGESTION_SOURCE_POLL_TIMEOUT.get(settings);
1088+
final int numProcessorThreads = INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING.get(settings);
1089+
10761090
return new IngestionSource.Builder(ingestionSourceType).setParams(ingestionSourceParams)
10771091
.setPointerInitReset(pointerInitReset)
10781092
.setErrorStrategy(errorStrategy)
10791093
.setMaxPollSize(maxPollSize)
10801094
.setPollTimeout(pollTimeout)
1095+
.setNumProcessorThreads(numProcessorThreads)
10811096
.build();
10821097
}
10831098
return null;

Diff for: server/src/main/java/org/opensearch/cluster/metadata/IngestionSource.java

+21-4
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818
import java.util.Objects;
1919

2020
import static org.opensearch.cluster.metadata.IndexMetadata.INGESTION_SOURCE_MAX_POLL_SIZE;
21+
import static org.opensearch.cluster.metadata.IndexMetadata.INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING;
2122
import static org.opensearch.cluster.metadata.IndexMetadata.INGESTION_SOURCE_POLL_TIMEOUT;
2223

2324
/**
@@ -31,21 +32,24 @@ public class IngestionSource {
3132
private final Map<String, Object> params;
3233
private final long maxPollSize;
3334
private final int pollTimeout;
35+
private int numProcessorThreads;
3436

3537
private IngestionSource(
3638
String type,
3739
PointerInitReset pointerInitReset,
3840
IngestionErrorStrategy.ErrorStrategy errorStrategy,
3941
Map<String, Object> params,
4042
long maxPollSize,
41-
int pollTimeout
43+
int pollTimeout,
44+
int numProcessorThreads
4245
) {
4346
this.type = type;
4447
this.pointerInitReset = pointerInitReset;
4548
this.params = params;
4649
this.errorStrategy = errorStrategy;
4750
this.maxPollSize = maxPollSize;
4851
this.pollTimeout = pollTimeout;
52+
this.numProcessorThreads = numProcessorThreads;
4953
}
5054

5155
public String getType() {
@@ -72,6 +76,10 @@ public int getPollTimeout() {
7276
return pollTimeout;
7377
}
7478

79+
public int getNumProcessorThreads() {
80+
return numProcessorThreads;
81+
}
82+
7583
@Override
7684
public boolean equals(Object o) {
7785
if (this == o) return true;
@@ -82,12 +90,13 @@ public boolean equals(Object o) {
8290
&& Objects.equals(errorStrategy, ingestionSource.errorStrategy)
8391
&& Objects.equals(params, ingestionSource.params)
8492
&& Objects.equals(maxPollSize, ingestionSource.maxPollSize)
85-
&& Objects.equals(pollTimeout, ingestionSource.pollTimeout);
93+
&& Objects.equals(pollTimeout, ingestionSource.pollTimeout)
94+
&& Objects.equals(numProcessorThreads, ingestionSource.numProcessorThreads);
8695
}
8796

8897
@Override
8998
public int hashCode() {
90-
return Objects.hash(type, pointerInitReset, params, errorStrategy, maxPollSize, pollTimeout);
99+
return Objects.hash(type, pointerInitReset, params, errorStrategy, maxPollSize, pollTimeout, numProcessorThreads);
91100
}
92101

93102
@Override
@@ -108,6 +117,8 @@ public String toString() {
108117
+ maxPollSize
109118
+ ", pollTimeout="
110119
+ pollTimeout
120+
+ ", numProcessorThreads="
121+
+ numProcessorThreads
111122
+ '}';
112123
}
113124

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

167179
public Builder(String type) {
168180
this.type = type;
@@ -206,8 +218,13 @@ public Builder setPollTimeout(int pollTimeout) {
206218
return this;
207219
}
208220

221+
public Builder setNumProcessorThreads(int numProcessorThreads) {
222+
this.numProcessorThreads = numProcessorThreads;
223+
return this;
224+
}
225+
209226
public IngestionSource build() {
210-
return new IngestionSource(type, pointerInitReset, errorStrategy, params, maxPollSize, pollTimeout);
227+
return new IngestionSource(type, pointerInitReset, errorStrategy, params, maxPollSize, pollTimeout, numProcessorThreads);
211228
}
212229

213230
}

Diff for: server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java

+3
Original file line numberDiff line numberDiff line change
@@ -269,6 +269,9 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
269269
IndexMetadata.INGESTION_SOURCE_POINTER_INIT_RESET_VALUE_SETTING,
270270
IndexMetadata.INGESTION_SOURCE_PARAMS_SETTING,
271271
IndexMetadata.INGESTION_SOURCE_ERROR_STRATEGY_SETTING,
272+
IndexMetadata.INGESTION_SOURCE_MAX_POLL_SIZE,
273+
IndexMetadata.INGESTION_SOURCE_POLL_TIMEOUT,
274+
IndexMetadata.INGESTION_SOURCE_NUM_PROCESSOR_THREADS_SETTING,
272275

273276
// validate that built-in similarities don't get redefined
274277
Setting.groupSetting("index.similarity.", (s) -> {

Diff for: server/src/main/java/org/opensearch/index/engine/IngestionEngine.java

+2-1
Original file line numberDiff line numberDiff line change
@@ -123,7 +123,8 @@ public void start() {
123123
ingestionErrorStrategy,
124124
initialPollerState,
125125
ingestionSource.getMaxPollSize(),
126-
ingestionSource.getPollTimeout()
126+
ingestionSource.getPollTimeout(),
127+
ingestionSource.getNumProcessorThreads()
127128
);
128129
streamPoller.start();
129130
}

Diff for: server/src/main/java/org/opensearch/indices/pollingingest/DefaultStreamPoller.java

+20-31
Original file line numberDiff line numberDiff line change
@@ -17,12 +17,11 @@
1717
import org.opensearch.index.Message;
1818
import org.opensearch.index.engine.IngestionEngine;
1919

20+
import java.util.Comparator;
2021
import java.util.List;
2122
import java.util.Locale;
2223
import java.util.Objects;
2324
import java.util.Set;
24-
import java.util.concurrent.ArrayBlockingQueue;
25-
import java.util.concurrent.BlockingQueue;
2625
import java.util.concurrent.ExecutorService;
2726
import java.util.concurrent.Executors;
2827

@@ -47,8 +46,6 @@ public class DefaultStreamPoller implements StreamPoller {
4746

4847
private ExecutorService consumerThread;
4948

50-
private ExecutorService processorThread;
51-
5249
// start of the batch, inclusive
5350
private IngestionShardPointer initialBatchStartPointer;
5451
private boolean includeBatchStartPointer = false;
@@ -58,16 +55,14 @@ public class DefaultStreamPoller implements StreamPoller {
5855

5956
private Set<IngestionShardPointer> persistedPointers;
6057

61-
private BlockingQueue<IngestionShardConsumer.ReadResult<? extends IngestionShardPointer, ? extends Message>> blockingQueue;
62-
63-
private MessageProcessorRunnable processorRunnable;
64-
6558
private final CounterMetric totalPolledCount = new CounterMetric();
6659

6760
// A pointer to the max persisted pointer for optimizing the check
6861
@Nullable
6962
private IngestionShardPointer maxPersistedPointer;
7063

64+
private PartitionedBlockingQueueContainer blockingQueueContainer;
65+
7166
public DefaultStreamPoller(
7267
IngestionShardPointer startPointer,
7368
Set<IngestionShardPointer> persistedPointers,
@@ -78,13 +73,14 @@ public DefaultStreamPoller(
7873
IngestionErrorStrategy errorStrategy,
7974
State initialState,
8075
long maxPollSize,
81-
int pollTimeout
76+
int pollTimeout,
77+
int numProcessorThreads
8278
) {
8379
this(
8480
startPointer,
8581
persistedPointers,
8682
consumer,
87-
new MessageProcessorRunnable(new ArrayBlockingQueue<>(100), ingestionEngine, errorStrategy),
83+
new PartitionedBlockingQueueContainer(numProcessorThreads, consumer.getShardId(), ingestionEngine, errorStrategy),
8884
resetState,
8985
resetValue,
9086
errorStrategy,
@@ -96,7 +92,7 @@ public DefaultStreamPoller(
9692
IngestionShardPointer startPointer,
9793
Set<IngestionShardPointer> persistedPointers,
9894
IngestionShardConsumer consumer,
99-
MessageProcessorRunnable processorRunnable,
95+
PartitionedBlockingQueueContainer blockingQueueContainer,
10096
ResetState resetState,
10197
String resetValue,
10298
IngestionErrorStrategy errorStrategy,
@@ -111,22 +107,13 @@ public DefaultStreamPoller(
111107
if (!this.persistedPointers.isEmpty()) {
112108
maxPersistedPointer = this.persistedPointers.stream().max(IngestionShardPointer::compareTo).get();
113109
}
114-
this.processorRunnable = processorRunnable;
115-
blockingQueue = processorRunnable.getBlockingQueue();
110+
this.blockingQueueContainer = blockingQueueContainer;
116111
this.consumerThread = Executors.newSingleThreadExecutor(
117112
r -> new Thread(
118113
r,
119114
String.format(Locale.ROOT, "stream-poller-consumer-%d-%d", consumer.getShardId(), System.currentTimeMillis())
120115
)
121116
);
122-
123-
// TODO: allow multiple threads for processing the messages in parallel
124-
this.processorThread = Executors.newSingleThreadExecutor(
125-
r -> new Thread(
126-
r,
127-
String.format(Locale.ROOT, "stream-poller-processor-%d-%d", consumer.getShardId(), System.currentTimeMillis())
128-
)
129-
);
130117
this.errorStrategy = errorStrategy;
131118
}
132119

@@ -144,7 +131,7 @@ public void start() {
144131
// when we start, we need to include the batch start pointer in the read for the first read
145132
includeBatchStartPointer = true;
146133
consumerThread.submit(this::startPoll);
147-
processorThread.submit(processorRunnable);
134+
blockingQueueContainer.startProcessorThreads();
148135
}
149136

150137
/**
@@ -242,7 +229,7 @@ private void processRecords(List<IngestionShardConsumer.ReadResult<? extends Ing
242229
continue;
243230
}
244231
totalPolledCount.inc();
245-
blockingQueue.put(result);
232+
blockingQueueContainer.add(result);
246233

247234
logger.debug(
248235
"Put message {} with pointer {} to the blocking queue",
@@ -322,10 +309,8 @@ public void close() {
322309
logger.error("Error in closing the poller of shard {}: {}", consumer.getShardId(), e);
323310
}
324311
}
325-
blockingQueue.clear();
326312
consumerThread.shutdown();
327-
// interrupts the processor
328-
processorThread.shutdownNow();
313+
blockingQueueContainer.close();
329314
logger.info("closed the poller of shard {}", consumer.getShardId());
330315
}
331316

@@ -343,19 +328,23 @@ public boolean isClosed() {
343328
* Returns the batch start pointer from where the poller can resume in case of shard recovery. The poller and
344329
* processor are decoupled in this implementation, and hence the latest pointer tracked by the processor acts as the
345330
* recovery/start point. In case the processor has not started tracking, then the initial batchStartPointer used by
346-
* the poller acts as the start point.
331+
* the poller acts as the start point. If multiple processor threads are used, the minimum shard pointer across
332+
* processors indicates the start point.
347333
*/
348334
@Override
349335
public IngestionShardPointer getBatchStartPointer() {
350-
IngestionShardPointer currentShardPointer = processorRunnable.getCurrentShardPointer();
351-
return currentShardPointer == null ? initialBatchStartPointer : currentShardPointer;
336+
return blockingQueueContainer.getCurrentShardPointers()
337+
.stream()
338+
.filter(Objects::nonNull)
339+
.min(Comparator.naturalOrder())
340+
.orElseGet(() -> initialBatchStartPointer);
352341
}
353342

354343
@Override
355344
public PollingIngestStats getStats() {
356345
PollingIngestStats.Builder builder = new PollingIngestStats.Builder();
357346
builder.setTotalPolledCount(totalPolledCount.count());
358-
builder.setTotalProcessedCount(processorRunnable.getStats().count());
347+
builder.setTotalProcessedCount(blockingQueueContainer.getTotalProcessedCount());
359348
return builder.build();
360349
}
361350

@@ -371,6 +360,6 @@ public IngestionErrorStrategy getErrorStrategy() {
371360
@Override
372361
public void updateErrorStrategy(IngestionErrorStrategy errorStrategy) {
373362
this.errorStrategy = errorStrategy;
374-
processorRunnable.setErrorStrategy(errorStrategy);
363+
blockingQueueContainer.updateErrorStrategy(errorStrategy);
375364
}
376365
}

0 commit comments

Comments
 (0)