Skip to content

Commit dc4722f

Browse files
Add error handling strategy to pull-based ingestion
Signed-off-by: Varun Bharadwaj <varunbharadwaj1995@gmail.com>
1 parent 8447737 commit dc4722f

File tree

14 files changed

+376
-28
lines changed

14 files changed

+376
-28
lines changed

plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPartitionConsumer.java

+5
Original file line numberDiff line numberDiff line change
@@ -123,6 +123,11 @@ public KafkaOffset nextPointer() {
123123
return new KafkaOffset(lastFetchedOffset + 1);
124124
}
125125

126+
@Override
127+
public KafkaOffset nextPointer(KafkaOffset pointer) {
128+
return new KafkaOffset(pointer.getOffset() + 1);
129+
}
130+
126131
@Override
127132
public IngestionShardPointer earliestPointer() {
128133
long startOffset = AccessController.doPrivileged(

server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java

+27-1
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,7 @@
7171
import org.opensearch.index.IndexModule;
7272
import org.opensearch.index.mapper.MapperService;
7373
import org.opensearch.index.seqno.SequenceNumbers;
74+
import org.opensearch.indices.pollingingest.IngestionErrorStrategy;
7475
import org.opensearch.indices.pollingingest.StreamPoller;
7576
import org.opensearch.indices.replication.SegmentReplicationSource;
7677
import org.opensearch.indices.replication.common.ReplicationType;
@@ -731,6 +732,30 @@ public void validate(final String value, final Map<Setting<?>, Object> settings)
731732
Property.Dynamic
732733
);
733734

735+
public static final String SETTING_INGESTION_SOURCE_ERROR_STRATEGY = "index.ingestion_source.error.strategy";
736+
public static final Setting<String> INGESTION_SOURCE_ERROR_STRATEGY_SETTING = Setting.simpleString(
737+
SETTING_INGESTION_SOURCE_ERROR_STRATEGY,
738+
IngestionErrorStrategy.ErrorStrategy.DROP.name(),
739+
new Setting.Validator<>() {
740+
741+
@Override
742+
public void validate(final String value) {
743+
try {
744+
IngestionErrorStrategy.ErrorStrategy.valueOf(value.toUpperCase(Locale.ROOT));
745+
} catch (IllegalArgumentException e) {
746+
throw new IllegalArgumentException("Invalid value for " + SETTING_INGESTION_SOURCE_ERROR_STRATEGY + " [" + value + "]");
747+
}
748+
}
749+
750+
@Override
751+
public void validate(final String value, final Map<Setting<?>, Object> settings) {
752+
validate(value);
753+
}
754+
},
755+
Property.IndexScope,
756+
Property.Dynamic
757+
);
758+
734759
public static final Setting.AffixSetting<Object> INGESTION_SOURCE_PARAMS_SETTING = Setting.prefixKeySetting(
735760
"index.ingestion_source.param.",
736761
key -> new Setting<>(key, "", (value) -> {
@@ -955,8 +980,9 @@ public IngestionSource getIngestionSource() {
955980
final String ingestionSourceType = INGESTION_SOURCE_TYPE_SETTING.get(settings);
956981
if (ingestionSourceType != null && !(NONE_INGESTION_SOURCE_TYPE.equals(ingestionSourceType))) {
957982
final String pointerInitReset = INGESTION_SOURCE_POINTER_INIT_RESET_SETTING.get(settings);
983+
final String errorStrategy = INGESTION_SOURCE_ERROR_STRATEGY_SETTING.get(settings);
958984
final Map<String, Object> ingestionSourceParams = INGESTION_SOURCE_PARAMS_SETTING.getAsMap(settings);
959-
return new IngestionSource(ingestionSourceType, pointerInitReset, ingestionSourceParams);
985+
return new IngestionSource(ingestionSourceType, pointerInitReset, errorStrategy, ingestionSourceParams);
960986
}
961987
return null;
962988
}

server/src/main/java/org/opensearch/cluster/metadata/IngestionSource.java

+22-3
Original file line numberDiff line numberDiff line change
@@ -20,12 +20,14 @@
2020
public class IngestionSource {
2121
private String type;
2222
private String pointerInitReset;
23+
private String errorStrategy;
2324
private Map<String, Object> params;
2425

25-
public IngestionSource(String type, String pointerInitReset, Map<String, Object> params) {
26+
public IngestionSource(String type, String pointerInitReset, String errorStrategy, Map<String, Object> params) {
2627
this.type = type;
2728
this.pointerInitReset = pointerInitReset;
2829
this.params = params;
30+
this.errorStrategy = errorStrategy;
2931
}
3032

3133
public String getType() {
@@ -36,6 +38,10 @@ public String getPointerInitReset() {
3638
return pointerInitReset;
3739
}
3840

41+
public String getErrorStrategy() {
42+
return errorStrategy;
43+
}
44+
3945
public Map<String, Object> params() {
4046
return params;
4147
}
@@ -47,16 +53,29 @@ public boolean equals(Object o) {
4753
IngestionSource ingestionSource = (IngestionSource) o;
4854
return Objects.equals(type, ingestionSource.type)
4955
&& Objects.equals(pointerInitReset, ingestionSource.pointerInitReset)
56+
&& Objects.equals(errorStrategy, ingestionSource.errorStrategy)
5057
&& Objects.equals(params, ingestionSource.params);
5158
}
5259

5360
@Override
5461
public int hashCode() {
55-
return Objects.hash(type, pointerInitReset, params);
62+
return Objects.hash(type, pointerInitReset, params, errorStrategy);
5663
}
5764

5865
@Override
5966
public String toString() {
60-
return "IngestionSource{" + "type='" + type + '\'' + ",pointer_init_reset='" + pointerInitReset + '\'' + ", params=" + params + '}';
67+
return "IngestionSource{"
68+
+ "type='"
69+
+ type
70+
+ '\''
71+
+ ",pointer_init_reset='"
72+
+ pointerInitReset
73+
+ '\''
74+
+ ",error_strategy='"
75+
+ errorStrategy
76+
+ '\''
77+
+ ", params="
78+
+ params
79+
+ '}';
6180
}
6281
}

server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java

+1
Original file line numberDiff line numberDiff line change
@@ -264,6 +264,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
264264
IndexMetadata.INGESTION_SOURCE_TYPE_SETTING,
265265
IndexMetadata.INGESTION_SOURCE_POINTER_INIT_RESET_SETTING,
266266
IndexMetadata.INGESTION_SOURCE_PARAMS_SETTING,
267+
IndexMetadata.INGESTION_SOURCE_ERROR_STRATEGY_SETTING,
267268

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

server/src/main/java/org/opensearch/index/IngestionShardConsumer.java

+5
Original file line numberDiff line numberDiff line change
@@ -72,6 +72,11 @@ public M getMessage() {
7272
*/
7373
T nextPointer();
7474

75+
/**
76+
* @return the immediate next pointer from the provided start pointer
77+
*/
78+
T nextPointer(T startPointer);
79+
7580
/**
7681
* @return the earliest pointer in the shard
7782
*/

server/src/main/java/org/opensearch/index/engine/IngestionEngine.java

+14-1
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@
5555
import org.opensearch.index.translog.TranslogManager;
5656
import org.opensearch.index.translog.TranslogStats;
5757
import org.opensearch.indices.pollingingest.DefaultStreamPoller;
58+
import org.opensearch.indices.pollingingest.IngestionErrorStrategy;
5859
import org.opensearch.indices.pollingingest.StreamPoller;
5960
import org.opensearch.search.suggest.completion.CompletionStats;
6061
import org.opensearch.threadpool.ThreadPool;
@@ -191,7 +192,19 @@ public void start() {
191192
resetState = StreamPoller.ResetState.NONE;
192193
}
193194

194-
streamPoller = new DefaultStreamPoller(startPointer, persistedPointers, ingestionShardConsumer, this, resetState);
195+
IngestionErrorStrategy.ErrorStrategy errorStrategy = IngestionErrorStrategy.ErrorStrategy.valueOf(
196+
ingestionSource.getErrorStrategy().toUpperCase(Locale.ROOT)
197+
);
198+
IngestionErrorStrategy ingestionErrorStratey = IngestionErrorStrategy.create(errorStrategy, ingestionSource.getType());
199+
200+
streamPoller = new DefaultStreamPoller(
201+
startPointer,
202+
persistedPointers,
203+
ingestionShardConsumer,
204+
this,
205+
resetState,
206+
ingestionErrorStratey
207+
);
195208
streamPoller.start();
196209
}
197210

Original file line numberDiff line numberDiff line change
@@ -0,0 +1,36 @@
1+
/*
2+
* SPDX-License-Identifier: Apache-2.0
3+
*
4+
* The OpenSearch Contributors require contributions made to
5+
* this file be licensed under the Apache-2.0 license or a
6+
* compatible open source license.
7+
*/
8+
9+
package org.opensearch.indices.pollingingest;
10+
11+
import org.apache.logging.log4j.LogManager;
12+
import org.apache.logging.log4j.Logger;
13+
14+
/**
15+
* This error handling strategy blocks on failures preventing processing of remaining updates in the ingestion source.
16+
*/
17+
public class BlockIngestionErrorStrategy implements IngestionErrorStrategy {
18+
private static final Logger logger = LogManager.getLogger(BlockIngestionErrorStrategy.class);
19+
private final String ingestionSource;
20+
21+
public BlockIngestionErrorStrategy(String ingestionSource) {
22+
this.ingestionSource = ingestionSource;
23+
}
24+
25+
@Override
26+
public void handleError(Throwable e, ErrorStage stage) {
27+
logger.error("Error processing update from {}: {}", ingestionSource, e);
28+
29+
// todo: record blocking update and emit metrics
30+
}
31+
32+
@Override
33+
public boolean shouldPauseIngestion(Throwable e, ErrorStage stage) {
34+
return true;
35+
}
36+
}

server/src/main/java/org/opensearch/indices/pollingingest/DefaultStreamPoller.java

+25-5
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,9 @@ public class DefaultStreamPoller implements StreamPoller {
5151
// start of the batch, inclusive
5252
private IngestionShardPointer batchStartPointer;
5353

54+
// indicates the last record successfully written to the blocking queue
55+
private IngestionShardPointer lastSuccessfulPointer;
56+
5457
private ResetState resetState;
5558

5659
private Set<IngestionShardPointer> persistedPointers;
@@ -63,19 +66,23 @@ public class DefaultStreamPoller implements StreamPoller {
6366
@Nullable
6467
private IngestionShardPointer maxPersistedPointer;
6568

69+
private IngestionErrorStrategy errorStrategy;
70+
6671
public DefaultStreamPoller(
6772
IngestionShardPointer startPointer,
6873
Set<IngestionShardPointer> persistedPointers,
6974
IngestionShardConsumer consumer,
7075
IngestionEngine ingestionEngine,
71-
ResetState resetState
76+
ResetState resetState,
77+
IngestionErrorStrategy errorStrategy
7278
) {
7379
this(
7480
startPointer,
7581
persistedPointers,
7682
consumer,
77-
new MessageProcessorRunnable(new ArrayBlockingQueue<>(100), ingestionEngine),
78-
resetState
83+
new MessageProcessorRunnable(new ArrayBlockingQueue<>(100), ingestionEngine, errorStrategy),
84+
resetState,
85+
errorStrategy
7986
);
8087
}
8188

@@ -84,7 +91,8 @@ public DefaultStreamPoller(
8491
Set<IngestionShardPointer> persistedPointers,
8592
IngestionShardConsumer consumer,
8693
MessageProcessorRunnable processorRunnable,
87-
ResetState resetState
94+
ResetState resetState,
95+
IngestionErrorStrategy errorStrategy
8896
) {
8997
this.consumer = Objects.requireNonNull(consumer);
9098
this.resetState = resetState;
@@ -109,6 +117,7 @@ public DefaultStreamPoller(
109117
String.format(Locale.ROOT, "stream-poller-processor-%d-%d", consumer.getShardId(), System.currentTimeMillis())
110118
)
111119
);
120+
this.errorStrategy = errorStrategy;
112121
}
113122

114123
@Override
@@ -188,6 +197,7 @@ protected void startPoll() {
188197
continue;
189198
}
190199
blockingQueue.put(result);
200+
lastSuccessfulPointer = result.getPointer();
191201
logger.debug(
192202
"Put message {} with pointer {} to the blocking queue",
193203
String.valueOf(result.getMessage().getPayload()),
@@ -197,8 +207,18 @@ protected void startPoll() {
197207
// update the batch start pointer to the next batch
198208
batchStartPointer = consumer.nextPointer();
199209
} catch (Throwable e) {
200-
// TODO better error handling
201210
logger.error("Error in polling the shard {}: {}", consumer.getShardId(), e);
211+
errorStrategy.handleError(e, IngestionErrorStrategy.ErrorStage.POLLING);
212+
213+
if (errorStrategy.shouldPauseIngestion(e, IngestionErrorStrategy.ErrorStage.POLLING)) {
214+
// Blocking error encountered. Pause poller to stop processing remaining updates.
215+
pause();
216+
} else {
217+
// Advance the batch start pointer to ignore the error and continue from next record
218+
batchStartPointer = lastSuccessfulPointer == null
219+
? consumer.nextPointer(batchStartPointer)
220+
: consumer.nextPointer(lastSuccessfulPointer);
221+
}
202222
}
203223
}
204224
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,37 @@
1+
/*
2+
* SPDX-License-Identifier: Apache-2.0
3+
*
4+
* The OpenSearch Contributors require contributions made to
5+
* this file be licensed under the Apache-2.0 license or a
6+
* compatible open source license.
7+
*/
8+
9+
package org.opensearch.indices.pollingingest;
10+
11+
import org.apache.logging.log4j.LogManager;
12+
import org.apache.logging.log4j.Logger;
13+
14+
/**
15+
* This error handling strategy drops failures and proceeds with remaining updates in the ingestion source.
16+
*/
17+
public class DropIngestionErrorStrategy implements IngestionErrorStrategy {
18+
private static final Logger logger = LogManager.getLogger(DropIngestionErrorStrategy.class);
19+
private final String ingestionSource;
20+
21+
public DropIngestionErrorStrategy(String ingestionSource) {
22+
this.ingestionSource = ingestionSource;
23+
}
24+
25+
@Override
26+
public void handleError(Throwable e, ErrorStage stage) {
27+
logger.error("Error processing update from {}: {}", ingestionSource, e);
28+
29+
// todo: record failed update stats and emit metrics
30+
}
31+
32+
@Override
33+
public boolean shouldPauseIngestion(Throwable e, ErrorStage stage) {
34+
return false;
35+
}
36+
37+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,49 @@
1+
/*
2+
* SPDX-License-Identifier: Apache-2.0
3+
*
4+
* The OpenSearch Contributors require contributions made to
5+
* this file be licensed under the Apache-2.0 license or a
6+
* compatible open source license.
7+
*/
8+
9+
package org.opensearch.indices.pollingingest;
10+
11+
/**
12+
* Defines the error handling strategy when an error is encountered either during polling records from ingestion source
13+
* or during processing the polled records.
14+
*/
15+
public interface IngestionErrorStrategy {
16+
17+
/**
18+
* Process and record the error.
19+
*/
20+
void handleError(Throwable e, ErrorStage stage);
21+
22+
/**
23+
* Indicates if ingestion must be paused, blocking further writes.
24+
*/
25+
boolean shouldPauseIngestion(Throwable e, ErrorStage stage);
26+
27+
static IngestionErrorStrategy create(ErrorStrategy errorStrategy, String ingestionSource) {
28+
switch (errorStrategy) {
29+
case BLOCK:
30+
return new BlockIngestionErrorStrategy(ingestionSource);
31+
case DROP:
32+
default:
33+
return new DropIngestionErrorStrategy(ingestionSource);
34+
}
35+
}
36+
37+
// Indicates available error handling strategies
38+
enum ErrorStrategy {
39+
DROP,
40+
BLOCK
41+
}
42+
43+
// Indicates different stages of encountered errors
44+
enum ErrorStage {
45+
POLLING,
46+
PROCESSING
47+
}
48+
49+
}

0 commit comments

Comments
 (0)