Skip to content

Commit e130253

Browse files
authored
Implement querying for existing documents to handle duplicates on condition or bulk request/operation failures (#5525)
Signed-off-by: Taylor Gray <[email protected]>
1 parent f277b5f commit e130253

File tree

11 files changed

+962
-19
lines changed

11 files changed

+962
-19
lines changed

data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/BulkOperationWrapper.java

+11-3
Original file line numberDiff line numberDiff line change
@@ -46,19 +46,23 @@ public class BulkOperationWrapper {
4646
private final BulkOperation bulkOperation;
4747
private final SerializedJson jsonNode;
4848

49+
private final String queryTerm;
50+
4951
public BulkOperationWrapper(final BulkOperation bulkOperation) {
50-
this(bulkOperation, null, null);
52+
this(bulkOperation, null, null, null);
5153
}
5254

53-
public BulkOperationWrapper(final BulkOperation bulkOperation, final EventHandle eventHandle, final SerializedJson jsonNode) {
55+
public BulkOperationWrapper(final BulkOperation bulkOperation, final EventHandle eventHandle, final SerializedJson jsonNode,
56+
final String queryTerm) {
5457
checkNotNull(bulkOperation);
5558
this.bulkOperation = bulkOperation;
5659
this.eventHandle = eventHandle;
5760
this.jsonNode = jsonNode;
61+
this.queryTerm = queryTerm;
5862
}
5963

6064
public BulkOperationWrapper(final BulkOperation bulkOperation, final EventHandle eventHandle) {
61-
this(bulkOperation, eventHandle, null);
65+
this(bulkOperation, eventHandle, null, null);
6266
}
6367

6468
public BulkOperation getBulkOperation() {
@@ -80,6 +84,10 @@ public Object getDocument() {
8084
return getValueFromConverter(BULK_OPERATION_TO_DOCUMENT_CONVERTERS);
8185
}
8286

87+
public String getTermValue() {
88+
return queryTerm;
89+
}
90+
8391
public String getIndex() {
8492
return getValueFromConverter(BULK_OPERATION_TO_INDEX_NAME_CONVERTERS);
8593
}

data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/BulkRetryStrategy.java

+50-5
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99
import com.google.common.collect.ImmutableList;
1010
import com.linecorp.armeria.client.retry.Backoff;
1111
import io.micrometer.core.instrument.Counter;
12+
import jakarta.json.stream.JsonParsingException;
1213
import org.opensearch.client.opensearch._types.ErrorCause;
1314
import org.opensearch.client.opensearch._types.OpenSearchException;
1415
import org.opensearch.client.opensearch.core.BulkRequest;
@@ -17,11 +18,13 @@
1718
import org.opensearch.dataprepper.metrics.PluginMetrics;
1819
import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.AccumulatingBulkRequest;
1920
import org.opensearch.dataprepper.plugins.sink.opensearch.dlq.FailedBulkOperation;
21+
import org.opensearch.dataprepper.plugins.sink.opensearch.index.ExistingDocumentQueryManager;
2022
import org.opensearch.rest.RestStatus;
2123
import org.slf4j.Logger;
2224
import org.slf4j.LoggerFactory;
2325

2426
import java.io.IOException;
27+
import java.net.SocketTimeoutException;
2528
import java.time.Duration;
2629
import java.util.Arrays;
2730
import java.util.HashSet;
@@ -99,6 +102,14 @@ public final class BulkRetryStrategy {
99102
RestStatus.REQUEST_TIMEOUT.getStatus()
100103
));
101104

105+
private static final Set<Integer> POTENTIAL_DUPLICATES_ERRORS = Set.of(
106+
RestStatus.INTERNAL_SERVER_ERROR.getStatus(),
107+
RestStatus.GATEWAY_TIMEOUT.getStatus());
108+
109+
private static final Set<Class<? extends Exception>> SOCKET_TIMEOUT_EXCEPTIONS = Set.of(
110+
SocketTimeoutException.class,
111+
JsonParsingException.class);
112+
102113
private final RequestFunction<AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest>, BulkResponse> requestFunction;
103114
private final BiConsumer<List<FailedBulkOperation>, Throwable> logFailure;
104115
private final PluginMetrics pluginMetrics;
@@ -119,6 +130,7 @@ public final class BulkRetryStrategy {
119130
private final Counter bulkRequestServerErrors;
120131
private final Counter documentsVersionConflictErrors;
121132
private final Counter documentsDuplicates;
133+
private final ExistingDocumentQueryManager existingDocumentQueryManager;
122134
private static final Logger LOG = LoggerFactory.getLogger(BulkRetryStrategy.class);
123135

124136
static class BulkOperationRequestResponse {
@@ -136,6 +148,7 @@ AccumulatingBulkRequest getBulkRequest() {
136148
BulkResponse getResponse() {
137149
return response;
138150
}
151+
Exception getException() { return exception; }
139152
String getExceptionMessage() {
140153
return exception != null ? exception.getMessage() : "-";
141154
}
@@ -147,7 +160,9 @@ public BulkRetryStrategy(final RequestFunction<AccumulatingBulkRequest<BulkOpera
147160
final int maxRetries,
148161
final Supplier<AccumulatingBulkRequest> bulkRequestSupplier,
149162
final String pipelineName,
150-
final String pluginName) {
163+
final String pluginName,
164+
final ExistingDocumentQueryManager existingDocumentQueryManager) {
165+
this.existingDocumentQueryManager = existingDocumentQueryManager;
151166
this.requestFunction = requestFunction;
152167
this.logFailure = logFailure;
153168
this.pluginMetrics = pluginMetrics;
@@ -193,16 +208,18 @@ public void execute(final AccumulatingBulkRequest bulkRequest) throws Interrupte
193208
final Backoff backoff = Backoff.exponential(INITIAL_DELAY_MS, MAXIMUM_DELAY_MS).withMaxAttempts(maxRetries);
194209
BulkOperationRequestResponse operationResponse;
195210
BulkResponse response = null;
211+
Exception exception = null;
196212
AccumulatingBulkRequest request = bulkRequest;
197213
int attempt = 1;
198214
do {
199-
operationResponse = handleRetry(request, response, attempt);
215+
operationResponse = handleRetry(request, response, attempt, exception);
200216
if (operationResponse != null) {
201217
final long delayMillis = backoff.nextDelayMillis(attempt++);
202218
String exceptionMessage = "";
203219
request = operationResponse.getBulkRequest();
204220
response = operationResponse.getResponse();
205221
exceptionMessage = operationResponse.getExceptionMessage();
222+
exception = operationResponse.getException();
206223
if (delayMillis < 0) {
207224
RuntimeException e = new RuntimeException(String.format("Number of retries reached the limit of max retries (configured value %d. Last exception message: %s)", maxRetries, exceptionMessage));
208225
handleFailures(request, null, e);
@@ -285,8 +302,11 @@ private void handleFailures(final AccumulatingBulkRequest<BulkOperationWrapper,
285302
bulkRequestFailedCounter.increment();
286303
}
287304

288-
private BulkOperationRequestResponse handleRetry(final AccumulatingBulkRequest request, final BulkResponse response, int retryCount) throws InterruptedException {
289-
final AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> bulkRequestForRetry = createBulkRequestForRetry(request, response);
305+
private BulkOperationRequestResponse handleRetry(final AccumulatingBulkRequest request,
306+
final BulkResponse response,
307+
int retryCount,
308+
final Exception previousException) throws InterruptedException {
309+
final AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> bulkRequestForRetry = createBulkRequestForRetry(request, response, previousException);
290310
if (bulkRequestForRetry.getOperationsCount() == 0) {
291311
return null;
292312
}
@@ -317,7 +337,14 @@ private BulkOperationRequestResponse handleRetry(final AccumulatingBulkRequest r
317337
}
318338

319339
private AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> createBulkRequestForRetry(
320-
final AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> request, final BulkResponse response) {
340+
final AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> request, final BulkResponse response, final Exception previousException) {
341+
if (shouldSendAllForQuerying(previousException)) {
342+
for (final BulkOperationWrapper bulkOperationWrapper : request.getOperations()) {
343+
existingDocumentQueryManager.addBulkOperation(bulkOperationWrapper);
344+
}
345+
return bulkRequestSupplier.get();
346+
}
347+
321348
if (response == null) {
322349
// first attempt or retry due to Exception
323350
return request;
@@ -329,6 +356,12 @@ private AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> createBulkReq
329356
BulkOperationWrapper bulkOperation =
330357
(BulkOperationWrapper)request.getOperationAt(index);
331358
if (isItemInError(bulkItemResponse)) {
359+
if (existingDocumentQueryManager != null && POTENTIAL_DUPLICATES_ERRORS.contains(bulkItemResponse.status())) {
360+
existingDocumentQueryManager.addBulkOperation(bulkOperation);
361+
index++;
362+
continue;
363+
}
364+
332365
if (!NON_RETRY_STATUS.contains(bulkItemResponse.status())) {
333366
requestToReissue.addOperation(bulkOperation);
334367
} else if (bulkItemResponse.error() != null && VERSION_CONFLICT_EXCEPTION_TYPE.equals(bulkItemResponse.error().type())) {
@@ -416,4 +449,16 @@ private boolean isDuplicateDocument(final BulkResponseItem item) {
416449
private Counter getDocumentStatusCounter(final int status) {
417450
return pluginMetrics.counterWithTags(DOCUMENT_STATUSES, "status", Integer.toString(status));
418451
}
452+
453+
private boolean shouldSendAllForQuerying(final Exception exception) {
454+
if (exception != null && existingDocumentQueryManager != null) {
455+
if (SOCKET_TIMEOUT_EXCEPTIONS.contains(exception.getClass())) {
456+
return true;
457+
}
458+
459+
return exception instanceof OpenSearchException && POTENTIAL_DUPLICATES_ERRORS.contains(((OpenSearchException) exception).status());
460+
}
461+
462+
return false;
463+
}
419464
}

data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSink.java

+68-7
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.opensearch.client.transport.TransportOptions;
2424
import org.opensearch.common.unit.ByteSizeUnit;
2525
import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier;
26+
import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory;
2627
import org.opensearch.dataprepper.expression.ExpressionEvaluationException;
2728
import org.opensearch.dataprepper.expression.ExpressionEvaluator;
2829
import org.opensearch.dataprepper.metrics.MetricNames;
@@ -61,6 +62,7 @@
6162
import org.opensearch.dataprepper.plugins.sink.opensearch.dlq.FailedDlqData;
6263
import org.opensearch.dataprepper.plugins.sink.opensearch.index.ClusterSettingsParser;
6364
import org.opensearch.dataprepper.plugins.sink.opensearch.index.DocumentBuilder;
65+
import org.opensearch.dataprepper.plugins.sink.opensearch.index.ExistingDocumentQueryManager;
6466
import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexManager;
6567
import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexManagerFactory;
6668
import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplateAPIWrapper;
@@ -78,11 +80,15 @@
7880
import java.nio.file.StandardOpenOption;
7981
import java.util.Collection;
8082
import java.util.Collections;
83+
import java.util.HashSet;
8184
import java.util.List;
8285
import java.util.Objects;
8386
import java.util.Optional;
87+
import java.util.Set;
8488
import java.util.StringJoiner;
8589
import java.util.concurrent.ConcurrentHashMap;
90+
import java.util.concurrent.ExecutorService;
91+
import java.util.concurrent.Executors;
8692
import java.util.concurrent.locks.ReentrantLock;
8793
import java.util.function.Function;
8894
import java.util.function.Supplier;
@@ -149,6 +155,12 @@ public class OpenSearchSink extends AbstractSink<Record<Event>> {
149155
private final ConcurrentHashMap<Long, Long> lastFlushTimeMap;
150156
private final PluginConfigObservable pluginConfigObservable;
151157

158+
private ExistingDocumentQueryManager existingDocumentQueryManager;
159+
160+
private final ExecutorService queryExecutorService;
161+
162+
private final int processWorkerThreads;
163+
152164
@DataPrepperPluginConstructor
153165
public OpenSearchSink(final PluginSetting pluginSetting,
154166
final SinkContext sinkContext,
@@ -158,6 +170,7 @@ public OpenSearchSink(final PluginSetting pluginSetting,
158170
final PluginConfigObservable pluginConfigObservable,
159171
final OpenSearchSinkConfig openSearchSinkConfiguration) {
160172
super(pluginSetting, Integer.MAX_VALUE, INITIALIZE_RETRY_WAIT_TIME_MS);
173+
this.processWorkerThreads = pipelineDescription.getNumberOfProcessWorkers();
161174
this.awsCredentialsSupplier = awsCredentialsSupplier;
162175
this.sinkContext = sinkContext != null ? sinkContext : new SinkContext(null, Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
163176
this.expressionEvaluator = expressionEvaluator;
@@ -190,6 +203,8 @@ public OpenSearchSink(final PluginSetting pluginSetting,
190203
this.lastFlushTimeMap = new ConcurrentHashMap<>();
191204
this.pluginConfigObservable = pluginConfigObservable;
192205
this.objectMapper = new ObjectMapper();
206+
this.queryExecutorService = openSearchSinkConfig.getIndexConfiguration().getQueryTerm() != null ?
207+
Executors.newSingleThreadExecutor(BackgroundThreadFactory.defaultExecutorThreadFactory("existing-document-query-manager")) : null;
193208

194209
final Optional<DlqConfiguration> dlqConfig = openSearchSinkConfig.getRetryConfiguration().getDlq();
195210
if (dlqConfig.isPresent()) {
@@ -233,6 +248,12 @@ private void doInitializeInternal() throws IOException {
233248
};
234249
openSearchClientRefresher = new OpenSearchClientRefresher(
235250
pluginMetrics, connectionConfiguration, clientFunction);
251+
252+
if (queryExecutorService != null) {
253+
existingDocumentQueryManager = new ExistingDocumentQueryManager(openSearchSinkConfig.getIndexConfiguration(), pluginMetrics, openSearchClient);
254+
queryExecutorService.submit(existingDocumentQueryManager);
255+
}
256+
236257
pluginConfigObservable.addPluginConfigObserver(
237258
newOpenSearchSinkConfig -> openSearchClientRefresher.update((OpenSearchSinkConfig) newOpenSearchSinkConfig));
238259
configuredIndexAlias = openSearchSinkConfig.getIndexConfiguration().getIndexAlias();
@@ -280,7 +301,8 @@ private void doInitializeInternal() throws IOException {
280301
maxRetries,
281302
bulkRequestSupplier,
282303
pipeline,
283-
PLUGIN_NAME);
304+
PLUGIN_NAME,
305+
openSearchSinkConfig.getIndexConfiguration().getQueryOnBulkFailures() ? existingDocumentQueryManager : null);
284306

285307
this.initialized = true;
286308
LOG.info("Initialized OpenSearch sink");
@@ -394,6 +416,22 @@ public void doOutput(final Collection<Record<Event>> records) {
394416
AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> bulkRequest = bulkRequestMap.get(threadId);
395417
long lastFlushTime = lastFlushTimeMap.get(threadId);
396418

419+
Set<BulkOperationWrapper> documentsReadyForIndexing = new HashSet<>();
420+
if (openSearchSinkConfig.getIndexConfiguration().getQueryTerm() != null) {
421+
documentsReadyForIndexing = existingDocumentQueryManager.getAndClearBulkOperationsReadyToIndex();
422+
}
423+
424+
425+
if (!documentsReadyForIndexing.isEmpty()) {
426+
LOG.info("Found {} documents ready for indexing from query manager", documentsReadyForIndexing.size());
427+
}
428+
429+
for (final BulkOperationWrapper bulkOperationWrapper : documentsReadyForIndexing) {
430+
bulkRequest = flushBatch(bulkRequest, bulkOperationWrapper, lastFlushTime);
431+
bulkRequest.addOperation(bulkOperationWrapper);
432+
}
433+
434+
397435
for (final Record<Event> record : records) {
398436
final Event event = record.getData();
399437
final SerializedJson document = getDocument(event);
@@ -465,13 +503,18 @@ public void doOutput(final Collection<Record<Event>> records) {
465503
continue;
466504
}
467505

468-
BulkOperationWrapper bulkOperationWrapper = new BulkOperationWrapper(bulkOperation, event.getEventHandle(), serializedJsonNode);
469-
final long estimatedBytesBeforeAdd = bulkRequest.estimateSizeInBytesWithDocument(bulkOperationWrapper);
470-
if (bulkSize >= 0 && estimatedBytesBeforeAdd >= bulkSize && bulkRequest.getOperationsCount() > 0) {
471-
flushBatch(bulkRequest);
472-
lastFlushTime = System.currentTimeMillis();
473-
bulkRequest = bulkRequestSupplier.get();
506+
final String queryTermKey = openSearchSinkConfig.getIndexConfiguration().getQueryTerm();
507+
final String termValue = queryTermKey != null ?
508+
event.get(queryTermKey, String.class) : null;
509+
BulkOperationWrapper bulkOperationWrapper = new BulkOperationWrapper(bulkOperation, event.getEventHandle(), serializedJsonNode, termValue);
510+
511+
if (openSearchSinkConfig.getIndexConfiguration().getQueryWhen() != null &&
512+
expressionEvaluator.evaluateConditional(openSearchSinkConfig.getIndexConfiguration().getQueryWhen(), event)) {
513+
existingDocumentQueryManager.addBulkOperation(bulkOperationWrapper);
514+
continue;
474515
}
516+
517+
bulkRequest = flushBatch(bulkRequest, bulkOperationWrapper, lastFlushTime);
475518
bulkRequest.addOperation(bulkOperationWrapper);
476519
}
477520

@@ -606,6 +649,10 @@ public void shutdown() {
606649
super.shutdown();
607650
closeFiles();
608651
openSearchClient.shutdown();
652+
if (queryExecutorService != null) {
653+
existingDocumentQueryManager.stop();
654+
queryExecutorService.shutdown();
655+
}
609656
}
610657

611658
private void maybeUpdateServerlessNetworkPolicy() {
@@ -652,4 +699,18 @@ private boolean isUsingDocumentFilters() {
652699
(sinkContext.getExcludeKeys() != null && !sinkContext.getExcludeKeys().isEmpty()) ||
653700
sinkContext.getTagsTargetKey() != null;
654701
}
702+
703+
private AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> flushBatch(
704+
final AccumulatingBulkRequest<BulkOperationWrapper, BulkRequest> bulkRequest,
705+
final BulkOperationWrapper bulkOperationWrapper,
706+
long lastFlushTime
707+
) {
708+
final long estimatedBytesBeforeAdd = bulkRequest.estimateSizeInBytesWithDocument(bulkOperationWrapper);
709+
if (bulkSize >= 0 && estimatedBytesBeforeAdd >= bulkSize && bulkRequest.getOperationsCount() > 0) {
710+
flushBatch(bulkRequest);
711+
lastFlushTime = System.currentTimeMillis();
712+
return bulkRequestSupplier.get();
713+
}
714+
return bulkRequest;
715+
}
655716
}

data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/configuration/OpenSearchSinkConfig.java

+5
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions;
1414
import org.opensearch.dataprepper.plugins.sink.opensearch.DistributionVersion;
1515
import org.opensearch.dataprepper.plugins.sink.opensearch.index.TemplateType;
16+
import org.opensearch.dataprepper.plugins.sink.opensearch.index.model.QueryForExistingDocumentConfiguration;
1617

1718
import java.util.List;
1819
import java.util.Map;
@@ -175,6 +176,10 @@ public boolean getEnableRequestCompression() {
175176
@JsonProperty("dlq")
176177
private DlqConfiguration dlq;
177178

179+
@Getter
180+
@JsonProperty("query_lookup")
181+
private QueryForExistingDocumentConfiguration queryExistingConfiguration;
182+
178183
@AssertTrue(message = "dlq_file option cannot be used along with dlq option")
179184
public boolean isDlqValid() {
180185
if (dlq != null) {

0 commit comments

Comments
 (0)