Skip to content
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

[Pull-based Ingestion] Add support for dynamically updating ingestion error handling strategy #17565

Open
wants to merge 2 commits into
base: main
Choose a base branch
from
Open
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
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.opensearch.indices.pollingingest.PollingIngestStats;
import org.opensearch.plugins.PluginInfo;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.transport.client.Requests;
import org.junit.Assert;

import java.util.List;
Expand Down Expand Up @@ -56,27 +57,14 @@ public void testPluginsAreInstalled() {
public void testKafkaIngestion() {
produceData("1", "name1", "24");
produceData("2", "name2", "20");

createIndex(
"test",
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
.put("ingestion_source.type", "kafka")
.put("ingestion_source.pointer.init.reset", "earliest")
.put("ingestion_source.param.topic", "test")
.put("ingestion_source.param.bootstrap_servers", kafka.getBootstrapServers())
.put("index.replication.type", "SEGMENT")
.build(),
"{\"properties\":{\"name\":{\"type\": \"text\"},\"age\":{\"type\": \"integer\"}}}}"
);
createIndexWithDefaultSettings(1, 0);

RangeQueryBuilder query = new RangeQueryBuilder("age").gte(21);
await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> {
refresh("test");
SearchResponse response = client().prepareSearch("test").setQuery(query).get();
refresh(indexName);
SearchResponse response = client().prepareSearch(indexName).setQuery(query).get();
assertThat(response.getHits().getTotalHits().value(), is(1L));
PollingIngestStats stats = client().admin().indices().prepareStats("test").get().getIndex("test").getShards()[0]
PollingIngestStats stats = client().admin().indices().prepareStats(indexName).get().getIndex(indexName).getShards()[0]
.getPollingIngestStats();
assertNotNull(stats);
assertThat(stats.getMessageProcessorStats().getTotalProcessedCount(), is(2L));
Expand Down Expand Up @@ -135,10 +123,16 @@ public void testKafkaIngestion_RewindByOffset() {
);

RangeQueryBuilder query = new RangeQueryBuilder("age").gte(0);
await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> {
await().atMost(1, TimeUnit.MINUTES).untilAsserted(() -> {
refresh("test_rewind_by_offset");
SearchResponse response = client().prepareSearch("test_rewind_by_offset").setQuery(query).get();
assertThat(response.getHits().getTotalHits().value(), is(1L));
});
}

public void testCloseIndex() throws Exception {
createIndexWithDefaultSettings(1, 0);
ensureGreen(indexName);
client().admin().indices().close(Requests.closeIndexRequest(indexName)).get();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer;
import org.opensearch.action.search.SearchResponse;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.settings.Settings;
import org.opensearch.plugins.Plugin;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.junit.After;
Expand All @@ -25,6 +27,7 @@
import java.util.List;
import java.util.Locale;
import java.util.Properties;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;

import org.testcontainers.containers.KafkaContainer;
Expand Down Expand Up @@ -108,4 +111,32 @@ protected void waitForSearchableDocs(long docCount, List<String> nodes) throws E
}
}, 1, TimeUnit.MINUTES);
}

protected void waitForState(Callable<Boolean> checkState) throws Exception {
assertBusy(() -> {
if (checkState.call() == false) {
fail("Provided state requirements not met");
}
}, 1, TimeUnit.MINUTES);
}

protected String getSettings(String indexName, String setting) {
return client().admin().indices().prepareGetSettings(indexName).get().getSetting(indexName, setting);
}

protected void createIndexWithDefaultSettings(int numShards, int numReplicas) {
createIndex(
indexName,
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShards)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, numReplicas)
.put("ingestion_source.type", "kafka")
.put("ingestion_source.pointer.init.reset", "earliest")
.put("ingestion_source.param.topic", topicName)
.put("ingestion_source.param.bootstrap_servers", kafka.getBootstrapServers())
.put("index.replication.type", "SEGMENT")
.build(),
"{\"properties\":{\"name\":{\"type\": \"text\"},\"age\":{\"type\": \"integer\"}}}}"
);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import org.opensearch.index.query.RangeQueryBuilder;
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.transport.client.Requests;

import java.nio.file.Path;
import java.util.Arrays;
Expand Down Expand Up @@ -46,20 +47,7 @@ public void testSegmentReplicationWithRemoteStore() throws Exception {

internalCluster().startClusterManagerOnlyNode();
final String nodeA = internalCluster().startDataOnlyNode();

createIndex(
indexName,
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)
.put("ingestion_source.type", "kafka")
.put("ingestion_source.pointer.init.reset", "earliest")
.put("ingestion_source.param.topic", topicName)
.put("ingestion_source.param.bootstrap_servers", kafka.getBootstrapServers())
.put("index.replication.type", "SEGMENT")
.build(),
mapping
);
createIndexWithDefaultSettings(1, 1);

ensureYellowAndNoInitializingShards(indexName);
final String nodeB = internalCluster().startDataOnlyNode();
Expand Down Expand Up @@ -117,6 +105,56 @@ public void testSegmentReplicationWithRemoteStore() throws Exception {
waitForSearchableDocs(6, Arrays.asList(nodeB, nodeC));
}

public void testCloseIndex() throws Exception {
produceData("1", "name1", "24");
produceData("2", "name2", "20");
internalCluster().startClusterManagerOnlyNode();
final String nodeA = internalCluster().startDataOnlyNode();
final String nodeB = internalCluster().startDataOnlyNode();

createIndexWithDefaultSettings(1, 1);
ensureGreen(indexName);
waitForSearchableDocs(2, Arrays.asList(nodeA, nodeB));
client().admin().indices().close(Requests.closeIndexRequest(indexName)).get();
}

public void testErrorStrategy() throws Exception {
produceData("1", "name1", "25");
// malformed message
produceData("2", "", "");
produceData("3", "name3", "25");

internalCluster().startClusterManagerOnlyNode();
final String node = internalCluster().startDataOnlyNode();

createIndex(
indexName,
Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)
.put("ingestion_source.type", "kafka")
.put("ingestion_source.error_strategy", "block")
.put("ingestion_source.pointer.init.reset", "earliest")
.put("ingestion_source.param.topic", topicName)
.put("ingestion_source.param.bootstrap_servers", kafka.getBootstrapServers())
.put("index.replication.type", "SEGMENT")
.build(),
"{\"properties\":{\"name\":{\"type\": \"text\"},\"age\":{\"type\": \"integer\"}}}}"
);

ensureGreen(indexName);
waitForState(() -> "block".equalsIgnoreCase(getSettings(indexName, "index.ingestion_source.error_strategy")));
waitForSearchableDocs(1, Arrays.asList(node));

client().admin()
.indices()
.prepareUpdateSettings(indexName)
.setSettings(Settings.builder().put("ingestion_source.error_strategy", "drop"))
.get();
waitForState(() -> "drop".equalsIgnoreCase(getSettings(indexName, "index.ingestion_source.error_strategy")));
waitForSearchableDocs(2, Arrays.asList(node));
}

private void verifyRemoteStoreEnabled(String node) {
GetSettingsResponse settingsResponse = client(node).admin().indices().prepareGetSettings(indexName).get();
String remoteStoreEnabled = settingsResponse.getIndexToSettings().get(indexName).get("index.remote_store.enabled");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -771,13 +771,17 @@ public Iterator<Setting<?>> settings() {
Property.Final
);

/**
* Defines the error strategy for pull-based ingestion.
*/
public static final String SETTING_INGESTION_SOURCE_ERROR_STRATEGY = "index.ingestion_source.error_strategy";
public static final Setting<IngestionErrorStrategy.ErrorStrategy> INGESTION_SOURCE_ERROR_STRATEGY_SETTING = new Setting<>(
SETTING_INGESTION_SOURCE_ERROR_STRATEGY,
IngestionErrorStrategy.ErrorStrategy.DROP.name(),
IngestionErrorStrategy.ErrorStrategy::parseFromString,
(errorStrategy) -> {},
Property.IndexScope
Property.IndexScope,
Property.Dynamic
);

public static final Setting.AffixSetting<Object> INGESTION_SOURCE_PARAMS_SETTING = Setting.prefixKeySetting(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@
super(engineConfig);
this.ingestionConsumerFactory = Objects.requireNonNull(ingestionConsumerFactory);
this.documentMapperForType = engineConfig.getDocumentMapperForTypeSupplier().get();

registerDynamicIndexSettingsHandlers();
}

/**
Expand Down Expand Up @@ -304,4 +304,21 @@
public PollingIngestStats pollingIngestStats() {
return streamPoller.getStats();
}

private void registerDynamicIndexSettingsHandlers() {
engineConfig.getIndexSettings()
.getScopedSettings()
.addSettingsUpdateConsumer(IndexMetadata.INGESTION_SOURCE_ERROR_STRATEGY_SETTING, this::updateErrorHandlingStrategy);
}

/**
* Handler for updating ingestion error strategy in the stream poller on dynamic index settings update.
*/
private void updateErrorHandlingStrategy(IngestionErrorStrategy.ErrorStrategy errorStrategy) {
IngestionErrorStrategy updatedIngestionErrorStrategy = IngestionErrorStrategy.create(

Check warning on line 318 in server/src/main/java/org/opensearch/index/engine/IngestionEngine.java

View check run for this annotation

Codecov / codecov/patch

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

Added line #L318 was not covered by tests
errorStrategy,
engineConfig.getIndexSettings().getIndexMetadata().getIngestionSource().getType()

Check warning on line 320 in server/src/main/java/org/opensearch/index/engine/IngestionEngine.java

View check run for this annotation

Codecov / codecov/patch

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

Added line #L320 was not covered by tests
);
streamPoller.updateErrorStrategy(updatedIngestionErrorStrategy);
}

Check warning on line 323 in server/src/main/java/org/opensearch/index/engine/IngestionEngine.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/index/engine/IngestionEngine.java#L322-L323

Added lines #L322 - L323 were not covered by tests
}
Original file line number Diff line number Diff line change
Expand Up @@ -238,6 +238,7 @@
import static org.opensearch.index.seqno.RetentionLeaseActions.RETAIN_ALL;
import static org.opensearch.index.seqno.SequenceNumbers.LOCAL_CHECKPOINT_KEY;
import static org.opensearch.index.seqno.SequenceNumbers.MAX_SEQ_NO;
import static org.opensearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED;
import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
import static org.opensearch.index.shard.IndexShard.ShardMigrationState.REMOTE_MIGRATING_SEEDED;
import static org.opensearch.index.shard.IndexShard.ShardMigrationState.REMOTE_MIGRATING_UNSEEDED;
Expand Down Expand Up @@ -451,7 +452,7 @@
aId,
indexSettings,
primaryTerm,
UNASSIGNED_SEQ_NO,
getInitialGlobalCheckpointForShard(indexSettings),
globalCheckpointListeners::globalCheckpointUpdated,
threadPool::absoluteTimeInMillis,
(retentionLeases, listener) -> retentionLeaseSyncer.sync(shardId, aId, getPendingPrimaryTerm(), retentionLeases, listener),
Expand Down Expand Up @@ -499,6 +500,19 @@
this.segmentReplicationStatsProvider = segmentReplicationStatsProvider;
}

/**
* By default, UNASSIGNED_SEQ_NO is used as the initial global checkpoint for new shard initialization. Ingestion
* source does not track sequence numbers explicitly and hence defaults to NO_OPS_PERFORMED for compatibility.
*
*/
private long getInitialGlobalCheckpointForShard(IndexSettings indexSettings) {
if (indexSettings.getIndexMetadata().useIngestionSource()) {
return NO_OPS_PERFORMED;

Check warning on line 510 in server/src/main/java/org/opensearch/index/shard/IndexShard.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/index/shard/IndexShard.java#L510

Added line #L510 was not covered by tests
}

return UNASSIGNED_SEQ_NO;
}

public ThreadPool getThreadPool() {
return this.threadPool;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ public void handleError(Throwable e, ErrorStage stage) {
}

@Override
public boolean shouldPauseIngestion(Throwable e, ErrorStage stage) {
return true;
public boolean shouldIgnoreError(Throwable e, ErrorStage stage) {
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ public class DefaultStreamPoller implements StreamPoller {
private volatile boolean started;
private volatile boolean closed;
private volatile boolean paused;
private volatile IngestionErrorStrategy errorStrategy;

private IngestionShardConsumer consumer;

Expand All @@ -67,8 +68,6 @@ public class DefaultStreamPoller implements StreamPoller {
@Nullable
private IngestionShardPointer maxPersistedPointer;

private IngestionErrorStrategy errorStrategy;

public DefaultStreamPoller(
IngestionShardPointer startPointer,
Set<IngestionShardPointer> persistedPointers,
Expand Down Expand Up @@ -231,14 +230,14 @@ protected void startPoll() {
logger.error("Error in polling the shard {}: {}", consumer.getShardId(), e);
errorStrategy.handleError(e, IngestionErrorStrategy.ErrorStage.POLLING);

if (errorStrategy.shouldPauseIngestion(e, IngestionErrorStrategy.ErrorStage.POLLING)) {
// Blocking error encountered. Pause poller to stop processing remaining updates.
pause();
} else {
if (errorStrategy.shouldIgnoreError(e, IngestionErrorStrategy.ErrorStage.POLLING)) {
// Advance the batch start pointer to ignore the error and continue from next record
batchStartPointer = lastSuccessfulPointer == null
? consumer.nextPointer(batchStartPointer)
: consumer.nextPointer(lastSuccessfulPointer);
} else {
// Blocking error encountered. Pause poller to stop processing remaining updates.
pause();
}
}
}
Expand Down Expand Up @@ -332,4 +331,15 @@ public PollingIngestStats getStats() {
public State getState() {
return state;
}

@Override
public IngestionErrorStrategy getErrorStrategy() {
return this.errorStrategy;
}

@Override
public void updateErrorStrategy(IngestionErrorStrategy errorStrategy) {
this.errorStrategy = errorStrategy;
processorRunnable.setErrorStrategy(errorStrategy);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@ public void handleError(Throwable e, ErrorStage stage) {
}

@Override
public boolean shouldPauseIngestion(Throwable e, ErrorStage stage) {
return false;
public boolean shouldIgnoreError(Throwable e, ErrorStage stage) {
return true;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,9 @@ public interface IngestionErrorStrategy {
void handleError(Throwable e, ErrorStage stage);

/**
* Indicates if ingestion must be paused, blocking further writes.
* Indicates if the error should be ignored.
*/
boolean shouldPauseIngestion(Throwable e, ErrorStage stage);
boolean shouldIgnoreError(Throwable e, ErrorStage stage);

static IngestionErrorStrategy create(ErrorStrategy errorStrategy, String ingestionSource) {
switch (errorStrategy) {
Expand Down
Loading
Loading