From b7cdcaab021fcb888f6f0904b58c6b6081419493 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 2 Apr 2024 13:18:31 +0800 Subject: [PATCH 01/17] [PoC][issues-12457] Support Batch Ingestion Signed-off-by: Liyun Xiu --- .../action/bulk/BatchIngestionOption.java | 43 +++ .../opensearch/action/bulk/BulkRequest.java | 20 ++ .../action/bulk/TransportBulkAction.java | 46 +-- .../bulk/TransportBulkActionHelper.java | 68 ++++ .../opensearch/ingest/CompoundProcessor.java | 30 +- .../org/opensearch/ingest/IngestService.java | 299 +++++++++++++++++- .../java/org/opensearch/ingest/Pipeline.java | 16 + .../java/org/opensearch/ingest/Processor.java | 34 ++ .../rest/action/document/RestBulkAction.java | 2 + .../bulk/TransportBulkActionHelperTests.java | 61 ++++ .../action/bulk/TransportBulkActionTests.java | 18 -- .../opensearch/ingest/IngestServiceTests.java | 6 +- 12 files changed, 571 insertions(+), 72 deletions(-) create mode 100644 server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java create mode 100644 server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java create mode 100644 server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java diff --git a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java new file mode 100644 index 0000000000000..0b6f923e3cf52 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java @@ -0,0 +1,43 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.bulk; + +import org.opensearch.common.annotation.PublicApi; + +/** + * An enum for batch ingestion option. + */ +@PublicApi(since = "2.14.0") +public enum BatchIngestionOption { + NONE("disabled"), + ENABLED("enabled"), + PARALLEL("parallel"); + + private final String value; + + public String getValue() { + return this.value; + } + + BatchIngestionOption(String value) { + this.value = value; + } + + static BatchIngestionOption from(String value) { + for (BatchIngestionOption option : values()) { + if (option.getValue().equals(value)) { + return option; + } + } + if ("".equals(value)) { + return NONE; + } + throw new IllegalArgumentException("Unknown value for batch ingestion option: [" + value + "]."); + } +} diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 47abd0337fcf9..d2a52058e2426 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -96,6 +96,8 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private String globalRouting; private String globalIndex; private Boolean globalRequireAlias; + private BatchIngestionOption batchIngestionOption; + private Integer maximumBatchSize; private long sizeInBytes = 0; @@ -346,6 +348,24 @@ public final BulkRequest timeout(TimeValue timeout) { return this; } + public final BulkRequest batchIngestionOption(String batchOption) { + this.batchIngestionOption = BatchIngestionOption.from(batchOption); + return this; + } + + public final BatchIngestionOption batchIngestionOption() { + return this.batchIngestionOption; + } + + public final BulkRequest maximumBatchSize(int size) { + this.maximumBatchSize = size; + return this; + } + + public final int maximumBatchSize() { + return this.maximumBatchSize; + } + /** * Note for internal callers (NOT high level rest client), * the global parameter setting is ignored when used with: diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java index 4a9b07c12821d..a8c043f57e473 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java @@ -208,24 +208,6 @@ public TransportBulkAction( this.tracer = tracer; } - /** - * Retrieves the {@link IndexRequest} from the provided {@link DocWriteRequest} for index or upsert actions. Upserts are - * modeled as {@link IndexRequest} inside the {@link UpdateRequest}. Ignores {@link org.opensearch.action.delete.DeleteRequest}'s - * - * @param docWriteRequest The request to find the {@link IndexRequest} - * @return the found {@link IndexRequest} or {@code null} if one can not be found. - */ - public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteRequest) { - IndexRequest indexRequest = null; - if (docWriteRequest instanceof IndexRequest) { - indexRequest = (IndexRequest) docWriteRequest; - } else if (docWriteRequest instanceof UpdateRequest) { - UpdateRequest updateRequest = (UpdateRequest) docWriteRequest; - indexRequest = updateRequest.docAsUpsert() ? updateRequest.doc() : updateRequest.upsertRequest(); - } - return indexRequest; - } - @Override protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { final boolean isOnlySystem = isOnlySystem(bulkRequest, clusterService.state().metadata().getIndicesLookup(), systemIndices); @@ -243,26 +225,12 @@ protected void doInternalExecute(Task task, BulkRequest bulkRequest, String exec final long startTime = relativeTime(); final AtomicArray responses = new AtomicArray<>(bulkRequest.requests.size()); - boolean hasIndexRequestsWithPipelines = false; final Metadata metadata = clusterService.state().getMetadata(); final Version minNodeVersion = clusterService.state().getNodes().getMinNodeVersion(); - for (DocWriteRequest actionRequest : bulkRequest.requests) { - IndexRequest indexRequest = getIndexWriteRequest(actionRequest); - if (indexRequest != null) { - // Each index request needs to be evaluated, because this method also modifies the IndexRequest - boolean indexRequestHasPipeline = IngestService.resolvePipelines(actionRequest, indexRequest, metadata); - hasIndexRequestsWithPipelines |= indexRequestHasPipeline; - } - if (actionRequest instanceof IndexRequest) { - IndexRequest ir = (IndexRequest) actionRequest; - ir.checkAutoIdWithOpTypeCreateSupportedByVersion(minNodeVersion); - if (ir.getAutoGeneratedTimestamp() != IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP) { - throw new IllegalArgumentException("autoGeneratedTimestamp should not be set externally"); - } - } - } + TransportBulkActionHelper.requestsValidation(bulkRequest.requests, minNodeVersion); + boolean hasIndexRequestsWithPipelines = TransportBulkActionHelper.resolvePipelines(bulkRequest.requests, metadata); if (hasIndexRequestsWithPipelines) { // this method (doExecute) will be called again, but with the bulk requests updated from the ingest node processing but // also with IngestService.NOOP_PIPELINE_NAME on each request. This ensures that this on the second time through this method, @@ -271,7 +239,7 @@ protected void doInternalExecute(Task task, BulkRequest bulkRequest, String exec if (Assertions.ENABLED) { final boolean arePipelinesResolved = bulkRequest.requests() .stream() - .map(TransportBulkAction::getIndexWriteRequest) + .map(TransportBulkActionHelper::getIndexWriteRequest) .filter(Objects::nonNull) .allMatch(IndexRequest::isPipelineResolved); assert arePipelinesResolved : bulkRequest; @@ -923,7 +891,8 @@ public boolean isForceExecution() { } }, bulkRequestModifier::markItemAsDropped, - executorName + executorName, + original ); } @@ -1000,7 +969,7 @@ ActionListener wrapActionListenerIfNeeded(long ingestTookInMillis, } synchronized void markItemAsDropped(int slot) { - IndexRequest indexRequest = getIndexWriteRequest(bulkRequest.requests().get(slot)); + IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(bulkRequest.requests().get(slot)); failedSlots.set(slot); final String id = indexRequest.id() == null ? DROPPED_ITEM_WITH_AUTO_GENERATED_ID : indexRequest.id(); itemResponses.add( @@ -1020,7 +989,7 @@ synchronized void markItemAsDropped(int slot) { } synchronized void markItemAsFailed(int slot, Exception e) { - IndexRequest indexRequest = getIndexWriteRequest(bulkRequest.requests().get(slot)); + IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(bulkRequest.requests().get(slot)); logger.debug( String.format( Locale.ROOT, @@ -1042,4 +1011,5 @@ synchronized void markItemAsFailed(int slot, Exception e) { } } + } diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java new file mode 100644 index 0000000000000..185baaac15040 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java @@ -0,0 +1,68 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.bulk; + +import org.opensearch.Version; +import org.opensearch.action.DocWriteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.ingest.IngestService; + +import java.util.List; + +/** + * A utility class for {@link TransportBulkAction}. + */ +public final class TransportBulkActionHelper { + private TransportBulkActionHelper() {} + + static void requestsValidation(final List> requests, final Version minNodeVersion) { + for (DocWriteRequest actionRequest : requests) { + if (actionRequest instanceof IndexRequest) { + IndexRequest ir = (IndexRequest) actionRequest; + ir.checkAutoIdWithOpTypeCreateSupportedByVersion(minNodeVersion); + if (ir.getAutoGeneratedTimestamp() != IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP) { + throw new IllegalArgumentException("autoGeneratedTimestamp should not be set externally"); + } + } + } + } + + /** + * Retrieves the {@link IndexRequest} from the provided {@link DocWriteRequest} for index or upsert actions. Upserts are + * modeled as {@link IndexRequest} inside the {@link UpdateRequest}. Ignores {@link org.opensearch.action.delete.DeleteRequest}'s + * + * @param docWriteRequest The request to find the {@link IndexRequest} + * @return the found {@link IndexRequest} or {@code null} if one can not be found. + */ + public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteRequest) { + IndexRequest indexRequest = null; + if (docWriteRequest instanceof IndexRequest) { + indexRequest = (IndexRequest) docWriteRequest; + } else if (docWriteRequest instanceof UpdateRequest) { + UpdateRequest updateRequest = (UpdateRequest) docWriteRequest; + indexRequest = updateRequest.docAsUpsert() ? updateRequest.doc() : updateRequest.upsertRequest(); + } + return indexRequest; + } + + static boolean resolvePipelines(final List> requests, Metadata metadata) { + boolean hasIndexRequestsWithPipelines = false; + for (DocWriteRequest actionRequest : requests) { + IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(actionRequest); + if (indexRequest != null) { + // Each index request needs to be evaluated, because this method also modifies the IndexRequest + boolean indexRequestHasPipeline = IngestService.resolvePipelines(actionRequest, indexRequest, metadata); + hasIndexRequestsWithPipelines |= indexRequestHasPipeline; + } + } + return hasIndexRequestsWithPipelines; + } +} diff --git a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java index a5f4870029e87..ca2a1fde036e3 100644 --- a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java @@ -43,6 +43,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.LongSupplier; import java.util.stream.Collectors; @@ -150,7 +151,34 @@ public void execute(IngestDocument ingestDocument, BiConsumer handler) { + @Override + public void batchExecute(List ingestDocuments, + Consumer>> handler) { + innerBatchExecute(0, ingestDocuments, handler); + } + + public void innerBatchExecute(int currentProcessor, List ingestDocuments, + Consumer>> handler) { + if (currentProcessor == processorsWithMetrics.size()) { + handler.accept(ingestDocuments.stream() + .map(t -> new Tuple(t, null)).collect(Collectors.toList())); + return; + } + Tuple processorWithMetric = processorsWithMetrics.get(currentProcessor); + final Processor processor = processorWithMetric.v1(); + final OperationMetrics metric = processorWithMetric.v2(); + final long startTimeInNanos = relativeTimeProvider.getAsLong(); + metric.before(); + processor.batchExecute(ingestDocuments, results -> { + long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); + metric.after(ingestTimeInMillis); + innerBatchExecute(currentProcessor + 1, + results.stream().map(Tuple::v1).collect(Collectors.toList()), handler); + }); + } + + void innerExecute(int currentProcessor, IngestDocument ingestDocument, + BiConsumer handler) { if (currentProcessor == processorsWithMetrics.size()) { handler.accept(ingestDocument, null); return; diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 2d4439e86461b..2ea8b72e5c551 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -39,7 +39,9 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.ResourceNotFoundException; import org.opensearch.action.DocWriteRequest; -import org.opensearch.action.bulk.TransportBulkAction; +import org.opensearch.action.bulk.BatchIngestionOption; +import org.opensearch.action.bulk.BulkRequest; +import org.opensearch.action.bulk.TransportBulkActionHelper; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.ingest.DeletePipelineRequest; import org.opensearch.action.ingest.PutPipelineRequest; @@ -93,6 +95,7 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.IntConsumer; +import java.util.stream.Collectors; /** * Holder class for several ingest related services. @@ -505,6 +508,9 @@ void validatePipeline(Map ingestInfos, PutPipelineReq ExceptionsHelper.rethrowAndSuppress(exceptions); } + /** + * Only used for testing + */ public void executeBulkRequest( int numberOfActionRequests, Iterable> actionRequests, @@ -513,7 +519,19 @@ public void executeBulkRequest( IntConsumer onDropped, String executorName ) { + executeBulkRequest(numberOfActionRequests, actionRequests, onFailure, onCompletion, + onDropped, executorName, new BulkRequest()); + } + public void executeBulkRequest( + int numberOfActionRequests, + Iterable> actionRequests, + BiConsumer onFailure, + BiConsumer onCompletion, + IntConsumer onDropped, + String executorName, + BulkRequest originalBulkRequst + ) { threadPool.executor(executorName).execute(new AbstractRunnable() { @Override @@ -525,9 +543,11 @@ public void onFailure(Exception e) { protected void doRun() { final Thread originalThread = Thread.currentThread(); final AtomicInteger counter = new AtomicInteger(numberOfActionRequests); + int i = 0; + List indexRequestWrappers = new ArrayList<>(); for (DocWriteRequest actionRequest : actionRequests) { - IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(actionRequest); + IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(actionRequest); if (indexRequest == null) { if (counter.decrementAndGet() == 0) { onCompletion.accept(originalThread, null); @@ -560,24 +580,214 @@ protected void doRun() { continue; } - executePipelines( - i, - pipelines.iterator(), - hasFinalPipeline, - indexRequest, - onDropped, - onFailure, - counter, - onCompletion, - originalThread - ); + indexRequestWrappers.add(new IndexRequestWrapper(i, indexRequest, pipelines, hasFinalPipeline)); i++; } + BatchIngestionOption batchOption = originalBulkRequst.batchIngestionOption(); + int batchSize = originalBulkRequst.maximumBatchSize(); + + if (batchOption == BatchIngestionOption.NONE || + (batchOption == BatchIngestionOption.ENABLED && indexRequestWrappers.size() == 1) || + (batchOption == BatchIngestionOption.ENABLED && batchSize == 1)) { + for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { + executePipelines(indexRequestWrapper.getSlot(), + indexRequestWrapper.getPipelines().iterator(), + indexRequestWrapper.isHasFinalPipeline(), + indexRequestWrapper.getIndexRequest(), + onDropped, onFailure, counter, onCompletion, originalThread); + } + } else { + List> batches = handleBatch(batchSize, indexRequestWrappers); + logger.info("batchSize: {}, batches: {}", batchSize, batches.size()); + for (List batch : batches) { + executePipelinesInBatchIndexRequests(batch.stream() + .map(IndexRequestWrapper::getSlot) + .collect(Collectors.toList()), + batch.get(0).getPipelines().iterator(), + batch.get(0).isHasFinalPipeline(), + batch.stream().map(IndexRequestWrapper::getIndexRequest).collect(Collectors.toList()), + onDropped, + onFailure, + counter, + onCompletion, + originalThread + ); + } + } } }); } + private List> handleBatch(int batchSize, List indexRequestWrappers) { + final Map> indexRequestsPerIndexMap = new HashMap<>(); + for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { + String index = indexRequestWrapper.getIndexRequest().index(); + indexRequestsPerIndexMap.putIfAbsent(index, new ArrayList<>()); + indexRequestsPerIndexMap.get(index).add(indexRequestWrapper); + } + List> batchedIndexRequests = new ArrayList<>(); + for (Map.Entry> indexRequestsPerIndex : indexRequestsPerIndexMap.entrySet()) { + for (int i = 0; i < indexRequestsPerIndex.getValue().size(); i += batchSize) { + batchedIndexRequests.add(new ArrayList<>( + indexRequestsPerIndex.getValue().subList(i, + i + Math.min(batchSize, indexRequestsPerIndex.getValue().size() - i)))); + } + } + return batchedIndexRequests; + } + + private static final class IndexRequestWrapper { + private final int slot; + private final IndexRequest indexRequest; + private final List pipelines; + private final boolean hasFinalPipeline; + + private IndexRequestWrapper(int slot, IndexRequest indexRequest, List pipelines, boolean hasFinalPipeline) { + this.slot = slot; + this.indexRequest = indexRequest; + this.pipelines = pipelines; + this.hasFinalPipeline = hasFinalPipeline; + } + + public int getSlot() { + return slot; + } + + public IndexRequest getIndexRequest() { + return indexRequest; + } + + public List getPipelines() { + return pipelines; + } + + public boolean isHasFinalPipeline() { + return hasFinalPipeline; + } + } + + private void executePipelinesInBatchIndexRequests( + final List slots, + final Iterator it, + final boolean hasFinalPipeline, + final List indexRequests, + final IntConsumer onDropped, + final BiConsumer onFailure, + final AtomicInteger counter, + final BiConsumer onCompletion, + final Thread originalThread + ) { + if (indexRequests.size() == 1) { + executePipelines(slots.get(0), it, hasFinalPipeline, indexRequests.get(0), onDropped, onFailure, counter, onCompletion, originalThread); + return; + } + while (it.hasNext()) { + final String pipelineId = it.next(); + try { + PipelineHolder holder = pipelines.get(pipelineId); + if (holder == null) { + throw new IllegalArgumentException("pipeline with id [" + pipelineId + "] does not exist"); + } + Pipeline pipeline = holder.pipeline; + String originalIndex = indexRequests.get(0).indices()[0]; + innerExecute(slots, indexRequests, pipeline, onDropped, exceptions -> { + for (int i = 0; i < indexRequests.size(); ++i) { + Exception e = exceptions.get(i); + IndexRequest indexRequest = indexRequests.get(i); + if (e != null) { + logger.debug( + () -> new ParameterizedMessage( + "failed to execute pipeline [{}] for document [{}/{}]", + pipelineId, + indexRequest.index(), + indexRequest.id() + ), + e + ); + onFailure.accept(slots.get(i), e); + } + } + + Iterator newIt = it; + boolean newHasFinalPipeline = hasFinalPipeline; + // indexRequests are grouped for the same index and same pipelines + String newIndex = indexRequests.get(0).indices()[0]; + + // handle index change case + if (Objects.equals(originalIndex, newIndex) == false) { + if (hasFinalPipeline && it.hasNext() == false) { + totalMetrics.failed(); + for (int slot : slots) { + onFailure.accept( + slot, + new IllegalStateException("final pipeline [" + pipelineId + "] can't change the target index") + ); + } + } else { + // Drain old it so it's not looped over + it.forEachRemaining($ -> {}); + for (IndexRequest indexRequest : indexRequests) { + indexRequest.isPipelineResolved(false); + resolvePipelines(null, indexRequest, state.metadata()); + if (IngestService.NOOP_PIPELINE_NAME.equals(indexRequest.getFinalPipeline()) == false) { + newIt = Collections.singleton(indexRequest.getFinalPipeline()).iterator(); + newHasFinalPipeline = true; + } else { + newIt = Collections.emptyIterator(); + } + } + } + } + + if (newIt.hasNext()) { + executePipelinesInBatchIndexRequests( + slots, + newIt, + newHasFinalPipeline, + indexRequests, + onDropped, + onFailure, + counter, + onCompletion, + originalThread + ); + } else { + if (counter.addAndGet(-indexRequests.size()) == 0) { + onCompletion.accept(originalThread, null); + } + assert counter.get() >= 0; + } + }); + } catch (Exception e) { + StringBuilder documentLogBuilder = new StringBuilder(); + for (int i = 0; i < indexRequests.size(); ++i) { + IndexRequest indexRequest = indexRequests.get(i); + documentLogBuilder.append(indexRequest.index()); + documentLogBuilder.append("/"); + documentLogBuilder.append(indexRequest.id()); + if (i < indexRequests.size() - 1) { + documentLogBuilder.append(", "); + } + onFailure.accept(slots.get(i), e); + } + logger.debug( + () -> new ParameterizedMessage( + "failed to execute pipeline [{}] for documents [{}]", + pipelineId, + documentLogBuilder.toString() + ), + e + ); + if (counter.addAndGet(-indexRequests.size()) == 0) { + onCompletion.accept(originalThread, null); + } + assert counter.get() >= 0; + break; + } + } + } + private void executePipelines( final int slot, final Iterator it, @@ -783,6 +993,69 @@ private void innerExecute( }); } + private void innerExecute( + List slots, + List indexRequests, + Pipeline pipeline, + IntConsumer itemDroppedHandler, + Consumer> handler + ) { + if (pipeline.getProcessors().isEmpty()) { + handler.accept(null); + return; + } + + long startTimeInNanos = System.nanoTime(); + // the pipeline specific stat holder may not exist and that is fine: + // (e.g. the pipeline may have been removed while we're ingesting a document + totalMetrics.before(); + List ingestDocuments = indexRequests.stream() + .map(IngestService::toIngestDocument) + .collect(Collectors.toList()); + pipeline.batchExecute(ingestDocuments, results -> { + long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos); + totalMetrics.after(ingestTimeInMillis); + List exceptions = Arrays.asList(new Exception[ingestDocuments.size()]); + for (int i = 0; i < results.size(); ++i) { + Tuple result = results.get(i); + Exception e = result.v2(); + IngestDocument ingestDocument = result.v1(); + IndexRequest indexRequest = indexRequests.get(i); + int slot = slots.get(i); + if (e != null) { + totalMetrics.failed(); + exceptions.set(i, e); + } else if (ingestDocument == null) { + itemDroppedHandler.accept(slot); + } else { + Map metadataMap = ingestDocument.extractMetadata(); + // it's fine to set all metadata fields all the time, as ingest document holds their starting values + // before ingestion, which might also get modified during ingestion. + indexRequests.get(i).index((String) metadataMap.get(IngestDocument.Metadata.INDEX)); + indexRequests.get(i).id((String) metadataMap.get(IngestDocument.Metadata.ID)); + indexRequests.get(i).routing((String) metadataMap.get(IngestDocument.Metadata.ROUTING)); + indexRequests.get(i).version(((Number) metadataMap.get(IngestDocument.Metadata.VERSION)).longValue()); + if (metadataMap.get(IngestDocument.Metadata.VERSION_TYPE) != null) { + indexRequests.get(i).versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.Metadata.VERSION_TYPE))); + } + if (metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO) != null) { + indexRequests.get(i).setIfSeqNo(((Number) metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO)).longValue()); + } + if (metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM) != null) { + indexRequests.get(i).setIfPrimaryTerm(((Number) metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM)).longValue()); + } + indexRequests.get(i).source(ingestDocument.getSourceAndMetadata(), indexRequest.getContentType()); + } + } + handler.accept(exceptions); + }); + } + + private static IngestDocument toIngestDocument(IndexRequest indexRequest) { + return new IngestDocument(indexRequest.index(), indexRequest.id(), indexRequest.routing(), + indexRequest.version(), indexRequest.versionType(), indexRequest.sourceAsMap()); + } + @Override public void applyClusterState(final ClusterChangedEvent event) { state = event.state(); diff --git a/server/src/main/java/org/opensearch/ingest/Pipeline.java b/server/src/main/java/org/opensearch/ingest/Pipeline.java index 2541cfbf4af77..6c29af670fbea 100644 --- a/server/src/main/java/org/opensearch/ingest/Pipeline.java +++ b/server/src/main/java/org/opensearch/ingest/Pipeline.java @@ -34,6 +34,7 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.common.Nullable; +import org.opensearch.common.collect.Tuple; import org.opensearch.common.metrics.OperationMetrics; import org.opensearch.script.ScriptService; @@ -43,6 +44,7 @@ import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.LongSupplier; /** @@ -201,4 +203,18 @@ public List flattenAllProcessors() { public OperationMetrics getMetrics() { return metrics; } + + public void batchExecute(List ingestDocuments, Consumer>> handler) { + final long startTimeInNanos = relativeTimeProvider.getAsLong(); + metrics.before(); + compoundProcessor.batchExecute(ingestDocuments, (List> results) -> { + long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); + metrics.after(ingestTimeInMillis); + // TODO: Check if any exception should make a failure. + if (results.stream().anyMatch(t -> t.v2() != null)) { + metrics.failed(); + } + handler.accept(results); + }); + } } diff --git a/server/src/main/java/org/opensearch/ingest/Processor.java b/server/src/main/java/org/opensearch/ingest/Processor.java index ecae1c139ea5e..e5168d79a073e 100644 --- a/server/src/main/java/org/opensearch/ingest/Processor.java +++ b/server/src/main/java/org/opensearch/ingest/Processor.java @@ -33,6 +33,8 @@ package org.opensearch.ingest; import org.opensearch.client.Client; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.env.Environment; import org.opensearch.index.analysis.AnalysisRegistry; @@ -40,7 +42,10 @@ import org.opensearch.script.ScriptService; import org.opensearch.threadpool.Scheduler; +import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Consumer; @@ -81,6 +86,35 @@ default void execute(IngestDocument ingestDocument, BiConsumer ingestDocuments, + Consumer>> handler) { + if (ingestDocuments.isEmpty()) { + handler.accept(Collections.emptyList()); + return; + } + int size = ingestDocuments.size(); + AtomicInteger counter = new AtomicInteger(size); + AtomicArray> results = new AtomicArray<>(size); + for (int i = 0; i < size; ++i) { + innerExecute(i, ingestDocuments.get(i), results, counter, handler); + } + } + + private void innerExecute(int slot, IngestDocument ingestDocument, + AtomicArray> results, AtomicInteger counter, + Consumer>> handler) { + execute(ingestDocument, (doc, ex) -> { + if (ex != null) { + results.set(slot, new Tuple<>(null, ex)); + } else { + results.set(slot, new Tuple<>(doc, null)); + } + if (counter.decrementAndGet() == 0) { + handler.accept(results.asList()); + } + }); + } + /** * Gets the type of a processor */ diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java index b046146707885..709a76bd9921a 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java @@ -97,6 +97,8 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC Boolean defaultRequireAlias = request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, null); bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); bulkRequest.setRefreshPolicy(request.param("refresh")); + bulkRequest.batchIngestionOption(request.param("batch_ingestion_option", "")); + bulkRequest.maximumBatchSize(request.paramAsInt("maximum_batch_size", 1)); bulkRequest.add( request.requiredContent(), defaultIndex, diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java new file mode 100644 index 0000000000000..f4953c3377bf4 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java @@ -0,0 +1,61 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.action.bulk; + +import org.opensearch.action.delete.DeleteRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.test.OpenSearchTestCase; + +import static java.util.Collections.emptyMap; + +public class TransportBulkActionHelperTests extends OpenSearchTestCase { + + public void testGetIndexWriteRequest() { + IndexRequest indexRequest = new IndexRequest("index").id("id1").source(emptyMap()); + UpdateRequest upsertRequest = new UpdateRequest("index", "id1").upsert(indexRequest).script(mockScript("1")); + UpdateRequest docAsUpsertRequest = new UpdateRequest("index", "id2").doc(indexRequest).docAsUpsert(true); + UpdateRequest scriptedUpsert = new UpdateRequest("index", "id2").upsert(indexRequest).script(mockScript("1")).scriptedUpsert(true); + + assertEquals(TransportBulkActionHelper.getIndexWriteRequest(indexRequest), indexRequest); + assertEquals(TransportBulkActionHelper.getIndexWriteRequest(upsertRequest), indexRequest); + assertEquals(TransportBulkActionHelper.getIndexWriteRequest(docAsUpsertRequest), indexRequest); + assertEquals(TransportBulkActionHelper.getIndexWriteRequest(scriptedUpsert), indexRequest); + + DeleteRequest deleteRequest = new DeleteRequest("index", "id"); + assertNull(TransportBulkActionHelper.getIndexWriteRequest(deleteRequest)); + + UpdateRequest badUpsertRequest = new UpdateRequest("index", "id1"); + assertNull(TransportBulkActionHelper.getIndexWriteRequest(badUpsertRequest)); + } +} diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java index 6bbd740df7f9c..93a7a0d55aeb9 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java @@ -207,24 +207,6 @@ public void testDeleteNonExistingDocExternalGteVersionCreatesIndex() throws Exce assertTrue(bulkAction.indexCreated); } - public void testGetIndexWriteRequest() throws Exception { - IndexRequest indexRequest = new IndexRequest("index").id("id1").source(emptyMap()); - UpdateRequest upsertRequest = new UpdateRequest("index", "id1").upsert(indexRequest).script(mockScript("1")); - UpdateRequest docAsUpsertRequest = new UpdateRequest("index", "id2").doc(indexRequest).docAsUpsert(true); - UpdateRequest scriptedUpsert = new UpdateRequest("index", "id2").upsert(indexRequest).script(mockScript("1")).scriptedUpsert(true); - - assertEquals(TransportBulkAction.getIndexWriteRequest(indexRequest), indexRequest); - assertEquals(TransportBulkAction.getIndexWriteRequest(upsertRequest), indexRequest); - assertEquals(TransportBulkAction.getIndexWriteRequest(docAsUpsertRequest), indexRequest); - assertEquals(TransportBulkAction.getIndexWriteRequest(scriptedUpsert), indexRequest); - - DeleteRequest deleteRequest = new DeleteRequest("index", "id"); - assertNull(TransportBulkAction.getIndexWriteRequest(deleteRequest)); - - UpdateRequest badUpsertRequest = new UpdateRequest("index", "id1"); - assertNull(TransportBulkAction.getIndexWriteRequest(badUpsertRequest)); - } - public void testProhibitAppendWritesInBackingIndices() throws Exception { String dataStreamName = "logs-foobar"; ClusterState clusterState = createDataStream(dataStreamName); diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 2edfe87387c92..72ac5e8a16733 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -40,6 +40,7 @@ import org.opensearch.action.DocWriteRequest; import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.TransportBulkAction; +import org.opensearch.action.bulk.TransportBulkActionHelper; import org.opensearch.action.delete.DeleteRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.ingest.DeletePipelineRequest; @@ -210,7 +211,8 @@ public void testExecuteIndexPipelineDoesNotExist() { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + new BulkRequest() ); assertTrue(failure.get()); @@ -1210,7 +1212,7 @@ public void testBulkRequestExecution() throws Exception { verify(requestItemErrorHandler, never()).accept(any(), any()); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); for (DocWriteRequest docWriteRequest : bulkRequest.requests()) { - IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(docWriteRequest); + IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(docWriteRequest); assertThat(indexRequest, notNullValue()); assertThat(indexRequest.getContentType(), equalTo(xContentType)); } From 8d4226d205127c4a7f23502ac7091c97619b7853 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Fri, 12 Apr 2024 10:12:30 +0800 Subject: [PATCH 02/17] Rewrite batch interface and handle error and metrics Signed-off-by: Liyun Xiu --- .../action/bulk/BatchIngestionOption.java | 5 +- .../opensearch/action/bulk/BulkRequest.java | 18 +++ .../common/metrics/OperationMetrics.java | 30 ++++ .../opensearch/ingest/CompoundProcessor.java | 94 +++++++++-- .../ingest/IngestDocumentWrapper.java | 42 +++++ .../org/opensearch/ingest/IngestService.java | 146 ++++++++---------- .../ingest/IngestServiceHelper.java | 68 ++++++++ .../java/org/opensearch/ingest/Pipeline.java | 16 +- .../java/org/opensearch/ingest/Processor.java | 27 ++-- 9 files changed, 324 insertions(+), 122 deletions(-) create mode 100644 server/src/main/java/org/opensearch/ingest/IngestDocumentWrapper.java create mode 100644 server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java diff --git a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java index 0b6f923e3cf52..e3e4621f868a5 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java +++ b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java @@ -13,11 +13,10 @@ /** * An enum for batch ingestion option. */ -@PublicApi(since = "2.14.0") +@PublicApi(since = "2.15.0") public enum BatchIngestionOption { NONE("disabled"), - ENABLED("enabled"), - PARALLEL("parallel"); + ENABLED("enabled"); private final String value; diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index d2a52058e2426..76932b2396a9e 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -348,20 +348,38 @@ public final BulkRequest timeout(TimeValue timeout) { return this; } + /** + * Convert string version of batch option and convert it to {@link BatchIngestionOption} + * @param batchOption a string input from request + * @return {@link BulkRequest} + */ public final BulkRequest batchIngestionOption(String batchOption) { this.batchIngestionOption = BatchIngestionOption.from(batchOption); return this; } + /** + * Get batch ingestion option + * @return {@link BatchIngestionOption} + */ public final BatchIngestionOption batchIngestionOption() { return this.batchIngestionOption; } + /** + * Set maximum batch size + * @param size maximum batch size from input + * @return {@link BulkRequest} + */ public final BulkRequest maximumBatchSize(int size) { this.maximumBatchSize = size; return this; } + /** + * Get maximum batch size + * @return maximum batch size + */ public final int maximumBatchSize() { return this.maximumBatchSize; } diff --git a/server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java b/server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java index 97fbbc2ce5cde..71c4a29f0f610 100644 --- a/server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java +++ b/server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java @@ -37,6 +37,14 @@ public void before() { current.incrementAndGet(); } + /** + * Invoke before the given operation begins in multiple items at the same time. + * @param n number of items + */ + public void beforeN(int n) { + current.addAndGet(n); + } + /** * Invoked upon completion (success or failure) of the given operation * @param currentTime elapsed time of the operation @@ -46,6 +54,18 @@ public void after(long currentTime) { time.inc(currentTime); } + /** + * Invoked upon completion (success or failure) of the given operation for multiple items. + * @param n number of items completed + * @param currentTime elapsed time of the operation + */ + public void afterN(int n, long currentTime) { + current.addAndGet(-n); + for (int i = 0; i < n; ++i) { + time.inc(currentTime); + } + } + /** * Invoked upon failure of the operation. */ @@ -53,6 +73,16 @@ public void failed() { failed.inc(); } + /** + * Invoked upon failure of the operation on multiple items. + * @param n number of items on operation. + */ + public void failedN(int n) { + for (int i = 0; i < n; ++i) { + failed.inc(); + } + } + public void add(OperationMetrics other) { // Don't try copying over current, since in-flight requests will be linked to the existing metrics instance. failed.inc(other.failed.count()); diff --git a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java index ca2a1fde036e3..b51189acc3360 100644 --- a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java @@ -39,9 +39,11 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.LongSupplier; @@ -152,28 +154,85 @@ public void execute(IngestDocument ingestDocument, BiConsumer ingestDocuments, - Consumer>> handler) { - innerBatchExecute(0, ingestDocuments, handler); + public void batchExecute(List ingestDocumentWrappers, + Consumer> handler) { + innerBatchExecute(0, ingestDocumentWrappers, handler); } - public void innerBatchExecute(int currentProcessor, List ingestDocuments, - Consumer>> handler) { + public void innerBatchExecute(int currentProcessor, List ingestDocumentWrappers, + Consumer> handler) { if (currentProcessor == processorsWithMetrics.size()) { - handler.accept(ingestDocuments.stream() - .map(t -> new Tuple(t, null)).collect(Collectors.toList())); + handler.accept(ingestDocumentWrappers); return; } Tuple processorWithMetric = processorsWithMetrics.get(currentProcessor); final Processor processor = processorWithMetric.v1(); final OperationMetrics metric = processorWithMetric.v2(); final long startTimeInNanos = relativeTimeProvider.getAsLong(); - metric.before(); - processor.batchExecute(ingestDocuments, results -> { - long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); - metric.after(ingestTimeInMillis); - innerBatchExecute(currentProcessor + 1, - results.stream().map(Tuple::v1).collect(Collectors.toList()), handler); + int size = ingestDocumentWrappers.size(); + metric.beforeN(size); + // Use synchronization to ensure batches are processed by processors in sequential order + AtomicInteger counter = new AtomicInteger(size); + List allResults = Collections.synchronizedList(new ArrayList<>()); + Map slotToWrapperMap = + createSlotIngestDocumentWrapperMap(ingestDocumentWrappers); + processor.batchExecute(ingestDocumentWrappers, results -> { + allResults.addAll(results); + if (counter.addAndGet(-results.size()) == 0) { + long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); + metric.afterN(allResults.size(), ingestTimeInMillis); + + List documentsDropped = new ArrayList<>(); + List documentsWithException = new ArrayList<>(); + List documentsToContinue = new ArrayList<>(); + int totalFailed = 0; + for (IngestDocumentWrapper resultDocumentWrapper : allResults) { + IngestDocumentWrapper originalDocumentWrapper = slotToWrapperMap.get(resultDocumentWrapper.getSlot()); + if (resultDocumentWrapper.getException() != null) { + ++totalFailed; + if (ignoreFailure) { + documentsToContinue.add(originalDocumentWrapper); + } else { + IngestProcessorException compoundProcessorException = + newCompoundProcessorException(resultDocumentWrapper.getException(), + processor, + originalDocumentWrapper.getIngestDocument() + ); + documentsWithException.add(new IngestDocumentWrapper(resultDocumentWrapper.getSlot(), + originalDocumentWrapper.getIngestDocument(), + compoundProcessorException + )); + } + } else { + if (resultDocumentWrapper.getIngestDocument() == null) { + documentsDropped.add(resultDocumentWrapper); + } else { + documentsToContinue.add(resultDocumentWrapper); + } + } + } + if (totalFailed > 0) { + metric.failedN(totalFailed); + } + if (!documentsDropped.isEmpty()) { + handler.accept(documentsDropped); + } + + if (!documentsToContinue.isEmpty()) { + innerBatchExecute(currentProcessor + 1, documentsToContinue, handler); + } + + if (!documentsWithException.isEmpty()) { + if (onFailureProcessors.isEmpty()) { + handler.accept(documentsWithException); + } else { + documentsWithException.forEach(doc -> executeOnFailureAsync(0, doc.getIngestDocument(), + (IngestProcessorException) doc.getException(), (result, ex) -> { + handler.accept(Collections.singletonList(new IngestDocumentWrapper(doc.getSlot(), result, ex))); + })); + } + } + } }); } @@ -294,4 +353,13 @@ static IngestProcessorException newCompoundProcessorException(Exception e, Proce return exception; } + private Map createSlotIngestDocumentWrapperMap( + List ingestDocumentWrappers) { + Map slotIngestDocumentWrapperMap = new HashMap<>(); + for (IngestDocumentWrapper ingestDocumentWrapper : ingestDocumentWrappers) { + slotIngestDocumentWrapperMap.put(ingestDocumentWrapper.getSlot(), ingestDocumentWrapper); + } + return slotIngestDocumentWrapperMap; + } + } diff --git a/server/src/main/java/org/opensearch/ingest/IngestDocumentWrapper.java b/server/src/main/java/org/opensearch/ingest/IngestDocumentWrapper.java new file mode 100644 index 0000000000000..6fb9f245f4996 --- /dev/null +++ b/server/src/main/java/org/opensearch/ingest/IngestDocumentWrapper.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.ingest; + +/** + * A IngestDocument wrapper including the slot of the IngestDocument in original IndexRequests. + * It also stores the exception happened during ingest process of the document. + */ +public final class IngestDocumentWrapper { + private final int slot; + private IngestDocument ingestDocument; + private Exception exception; + + public IngestDocumentWrapper(int slot, IngestDocument ingestDocument, Exception ex) { + this.slot = slot; + this.ingestDocument = ingestDocument; + this.exception = ex; + } + + public int getSlot() { + return this.slot; + } + + public IngestDocument getIngestDocument() { + return this.ingestDocument; + } + + public Exception getException() { + return this.exception; + } + + public void update(IngestDocument result, Exception ex) { + this.ingestDocument = result; + this.exception = ex; + } +} diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 2ea8b72e5c551..ac9e03a24a5c3 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -584,24 +584,14 @@ protected void doRun() { i++; } + BatchIngestionOption batchOption = originalBulkRequst.batchIngestionOption(); int batchSize = originalBulkRequst.maximumBatchSize(); - - if (batchOption == BatchIngestionOption.NONE || - (batchOption == BatchIngestionOption.ENABLED && indexRequestWrappers.size() == 1) || - (batchOption == BatchIngestionOption.ENABLED && batchSize == 1)) { - for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { - executePipelines(indexRequestWrapper.getSlot(), - indexRequestWrapper.getPipelines().iterator(), - indexRequestWrapper.isHasFinalPipeline(), - indexRequestWrapper.getIndexRequest(), - onDropped, onFailure, counter, onCompletion, originalThread); - } - } else { + if (shouldExecutePipelineInBatch(batchOption, indexRequestWrappers.size(), batchSize)) { List> batches = handleBatch(batchSize, indexRequestWrappers); logger.info("batchSize: {}, batches: {}", batchSize, batches.size()); for (List batch : batches) { - executePipelinesInBatchIndexRequests(batch.stream() + executePipelinesInBatchRequests(batch.stream() .map(IndexRequestWrapper::getSlot) .collect(Collectors.toList()), batch.get(0).getPipelines().iterator(), @@ -614,11 +604,23 @@ protected void doRun() { originalThread ); } + } else { + for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { + executePipelines(indexRequestWrapper.getSlot(), + indexRequestWrapper.getPipelines().iterator(), + indexRequestWrapper.isHasFinalPipeline(), + indexRequestWrapper.getIndexRequest(), + onDropped, onFailure, counter, onCompletion, originalThread); + } } } }); } + private boolean shouldExecutePipelineInBatch(BatchIngestionOption batchOption, int documentSize, int batchSize) { + return batchOption == BatchIngestionOption.ENABLED && documentSize > 1 && batchSize > 1; + } + private List> handleBatch(int batchSize, List indexRequestWrappers) { final Map> indexRequestsPerIndexMap = new HashMap<>(); for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { @@ -667,7 +669,7 @@ public boolean isHasFinalPipeline() { } } - private void executePipelinesInBatchIndexRequests( + private void executePipelinesInBatchRequests( final List slots, final Iterator it, final boolean hasFinalPipeline, @@ -691,11 +693,12 @@ private void executePipelinesInBatchIndexRequests( } Pipeline pipeline = holder.pipeline; String originalIndex = indexRequests.get(0).indices()[0]; - innerExecute(slots, indexRequests, pipeline, onDropped, exceptions -> { - for (int i = 0; i < indexRequests.size(); ++i) { - Exception e = exceptions.get(i); - IndexRequest indexRequest = indexRequests.get(i); - if (e != null) { + Map slotIndexRequestMap = IngestServiceHelper.createSlotIndexRequestMap(slots, + indexRequests); + innerExecute(slots, indexRequests, pipeline, onDropped, results -> { + for (int i = 0; i < results.size(); ++i) { + if (results.get(i).getException() != null) { + IndexRequest indexRequest = slotIndexRequestMap.get(results.get(i).getSlot()); logger.debug( () -> new ParameterizedMessage( "failed to execute pipeline [{}] for document [{}/{}]", @@ -703,9 +706,9 @@ private void executePipelinesInBatchIndexRequests( indexRequest.index(), indexRequest.id() ), - e + results.get(i).getException() ); - onFailure.accept(slots.get(i), e); + onFailure.accept(slots.get(i), results.get(i).getException()); } } @@ -741,7 +744,7 @@ private void executePipelinesInBatchIndexRequests( } if (newIt.hasNext()) { - executePipelinesInBatchIndexRequests( + executePipelinesInBatchRequests( slots, newIt, newHasFinalPipeline, @@ -753,7 +756,7 @@ private void executePipelinesInBatchIndexRequests( originalThread ); } else { - if (counter.addAndGet(-indexRequests.size()) == 0) { + if (counter.addAndGet(-results.size()) == 0) { onCompletion.accept(originalThread, null); } assert counter.get() >= 0; @@ -971,23 +974,7 @@ private void innerExecute( itemDroppedHandler.accept(slot); handler.accept(null); } else { - Map metadataMap = ingestDocument.extractMetadata(); - // it's fine to set all metadata fields all the time, as ingest document holds their starting values - // before ingestion, which might also get modified during ingestion. - indexRequest.index((String) metadataMap.get(IngestDocument.Metadata.INDEX)); - indexRequest.id((String) metadataMap.get(IngestDocument.Metadata.ID)); - indexRequest.routing((String) metadataMap.get(IngestDocument.Metadata.ROUTING)); - indexRequest.version(((Number) metadataMap.get(IngestDocument.Metadata.VERSION)).longValue()); - if (metadataMap.get(IngestDocument.Metadata.VERSION_TYPE) != null) { - indexRequest.versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.Metadata.VERSION_TYPE))); - } - if (metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO) != null) { - indexRequest.setIfSeqNo(((Number) metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO)).longValue()); - } - if (metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM) != null) { - indexRequest.setIfPrimaryTerm(((Number) metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM)).longValue()); - } - indexRequest.source(ingestDocument.getSourceAndMetadata(), indexRequest.getContentType()); + IngestServiceHelper.updateIndexRequestWithIngestDocument(indexRequest, ingestDocument); handler.accept(null); } }); @@ -998,64 +985,59 @@ private void innerExecute( List indexRequests, Pipeline pipeline, IntConsumer itemDroppedHandler, - Consumer> handler + Consumer> handler ) { if (pipeline.getProcessors().isEmpty()) { handler.accept(null); return; } + int size = indexRequests.size(); long startTimeInNanos = System.nanoTime(); // the pipeline specific stat holder may not exist and that is fine: // (e.g. the pipeline may have been removed while we're ingesting a document - totalMetrics.before(); - List ingestDocuments = indexRequests.stream() - .map(IngestService::toIngestDocument) - .collect(Collectors.toList()); - pipeline.batchExecute(ingestDocuments, results -> { - long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos); - totalMetrics.after(ingestTimeInMillis); - List exceptions = Arrays.asList(new Exception[ingestDocuments.size()]); - for (int i = 0; i < results.size(); ++i) { - Tuple result = results.get(i); - Exception e = result.v2(); - IngestDocument ingestDocument = result.v1(); - IndexRequest indexRequest = indexRequests.get(i); - int slot = slots.get(i); - if (e != null) { - totalMetrics.failed(); - exceptions.set(i, e); - } else if (ingestDocument == null) { - itemDroppedHandler.accept(slot); - } else { - Map metadataMap = ingestDocument.extractMetadata(); - // it's fine to set all metadata fields all the time, as ingest document holds their starting values - // before ingestion, which might also get modified during ingestion. - indexRequests.get(i).index((String) metadataMap.get(IngestDocument.Metadata.INDEX)); - indexRequests.get(i).id((String) metadataMap.get(IngestDocument.Metadata.ID)); - indexRequests.get(i).routing((String) metadataMap.get(IngestDocument.Metadata.ROUTING)); - indexRequests.get(i).version(((Number) metadataMap.get(IngestDocument.Metadata.VERSION)).longValue()); - if (metadataMap.get(IngestDocument.Metadata.VERSION_TYPE) != null) { - indexRequests.get(i).versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.Metadata.VERSION_TYPE))); - } - if (metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO) != null) { - indexRequests.get(i).setIfSeqNo(((Number) metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO)).longValue()); + totalMetrics.beforeN(size); + List ingestDocumentWrappers = new ArrayList<>(); + Map slotToindexRequestMap = new HashMap<>(); + for (int i = 0; i < slots.size(); ++i) { + slotToindexRequestMap.put(slots.get(i), indexRequests.get(i)); + ingestDocumentWrappers.add(IngestServiceHelper.toIngestDocumentWrapper(slots.get(i), indexRequests.get(i))); + } + AtomicInteger counter = new AtomicInteger(size); + List allResults = Collections.synchronizedList(new ArrayList<>()); + pipeline.batchExecute(ingestDocumentWrappers, results -> { + allResults.addAll(results); + if (counter.addAndGet(-results.size()) == 0) { + long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos); + totalMetrics.afterN(size, ingestTimeInMillis); + List succeeded = new ArrayList<>(); + List dropped = new ArrayList<>(); + List exceptions = new ArrayList<>(); + for (IngestDocumentWrapper result : results) { + if (result.getException() != null) { + exceptions.add(result); + } else if (result.getIngestDocument() == null) { + dropped.add(result); + } else { + succeeded.add(result); } - if (metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM) != null) { - indexRequests.get(i).setIfPrimaryTerm(((Number) metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM)).longValue()); + } + if (!exceptions.isEmpty()) { + totalMetrics.failedN(exceptions.size()); + } else if (!dropped.isEmpty()) { + dropped.forEach(t -> itemDroppedHandler.accept(t.getSlot())); + } else { + for (IngestDocumentWrapper ingestDocumentWrapper : succeeded) { + IngestServiceHelper.updateIndexRequestWithIngestDocument( + slotToindexRequestMap.get(ingestDocumentWrapper.getSlot()), + ingestDocumentWrapper.getIngestDocument()); } - indexRequests.get(i).source(ingestDocument.getSourceAndMetadata(), indexRequest.getContentType()); } + handler.accept(results); } - handler.accept(exceptions); }); } - private static IngestDocument toIngestDocument(IndexRequest indexRequest) { - return new IngestDocument(indexRequest.index(), indexRequest.id(), indexRequest.routing(), - indexRequest.version(), indexRequest.versionType(), indexRequest.sourceAsMap()); - } - @Override public void applyClusterState(final ClusterChangedEvent event) { state = event.state(); diff --git a/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java b/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java new file mode 100644 index 0000000000000..482a821feb288 --- /dev/null +++ b/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java @@ -0,0 +1,68 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.ingest; + +import org.opensearch.action.index.IndexRequest; +import org.opensearch.index.VersionType; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This is a helper class for static functions which can live outside IngestService. + */ +public class IngestServiceHelper { + private IngestServiceHelper() {} + + public static void updateIndexRequestWithIngestDocument(IndexRequest indexRequest, IngestDocument ingestDocument) { + if (ingestDocument == null || indexRequest == null) { + return; + } + Map metadataMap = ingestDocument.extractMetadata(); + // it's fine to set all metadata fields all the time, as ingest document holds their starting values + // before ingestion, which might also get modified during ingestion. + indexRequest.index((String) metadataMap.get(IngestDocument.Metadata.INDEX)); + indexRequest.id((String) metadataMap.get(IngestDocument.Metadata.ID)); + indexRequest.routing((String) metadataMap.get(IngestDocument.Metadata.ROUTING)); + indexRequest.version(((Number) metadataMap.get(IngestDocument.Metadata.VERSION)).longValue()); + if (metadataMap.get(IngestDocument.Metadata.VERSION_TYPE) != null) { + indexRequest.versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.Metadata.VERSION_TYPE))); + } + if (metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO) != null) { + indexRequest.setIfSeqNo(((Number) metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO)).longValue()); + } + if (metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM) != null) { + indexRequest.setIfPrimaryTerm(((Number) metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM)).longValue()); + } + indexRequest.source(ingestDocument.getSourceAndMetadata(), indexRequest.getContentType()); + } + + + public static IngestDocument toIngestDocument(IndexRequest indexRequest) { + return new IngestDocument(indexRequest.index(), indexRequest.id(), indexRequest.routing(), + indexRequest.version(), indexRequest.versionType(), indexRequest.sourceAsMap()); + } + + public static IngestDocumentWrapper toIngestDocumentWrapper(int slot, IndexRequest indexRequest) { + return new IngestDocumentWrapper(slot, toIngestDocument(indexRequest), null); + } + + public static Map createSlotIndexRequestMap(List slots, + List indexRequests) { + Map slotIndexRequestMap = new HashMap<>(); + for (int i = 0; i < slots.size(); ++i) { + slotIndexRequestMap.put(slots.get(i), indexRequests.get(i)); + } + return slotIndexRequestMap; + } +} + + + diff --git a/server/src/main/java/org/opensearch/ingest/Pipeline.java b/server/src/main/java/org/opensearch/ingest/Pipeline.java index 6c29af670fbea..105da5f72632f 100644 --- a/server/src/main/java/org/opensearch/ingest/Pipeline.java +++ b/server/src/main/java/org/opensearch/ingest/Pipeline.java @@ -204,16 +204,16 @@ public OperationMetrics getMetrics() { return metrics; } - public void batchExecute(List ingestDocuments, Consumer>> handler) { + public void batchExecute(List ingestDocumentWrappers, Consumer> handler) { final long startTimeInNanos = relativeTimeProvider.getAsLong(); - metrics.before(); - compoundProcessor.batchExecute(ingestDocuments, (List> results) -> { + int size = ingestDocumentWrappers.size(); + metrics.beforeN(size); + compoundProcessor.batchExecute(ingestDocumentWrappers, results -> { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); - metrics.after(ingestTimeInMillis); - // TODO: Check if any exception should make a failure. - if (results.stream().anyMatch(t -> t.v2() != null)) { - metrics.failed(); - } + metrics.afterN(results.size(), ingestTimeInMillis); + + int failedCount = (int) results.stream().filter(t -> t.getException() != null).count(); + metrics.failedN(failedCount); handler.accept(results); }); } diff --git a/server/src/main/java/org/opensearch/ingest/Processor.java b/server/src/main/java/org/opensearch/ingest/Processor.java index e5168d79a073e..1a2d8f734619a 100644 --- a/server/src/main/java/org/opensearch/ingest/Processor.java +++ b/server/src/main/java/org/opensearch/ingest/Processor.java @@ -33,7 +33,6 @@ package org.opensearch.ingest; import org.opensearch.client.Client; -import org.opensearch.common.collect.Tuple; import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.env.Environment; @@ -86,29 +85,25 @@ default void execute(IngestDocument ingestDocument, BiConsumer ingestDocuments, - Consumer>> handler) { - if (ingestDocuments.isEmpty()) { + default void batchExecute(List ingestDocumentWrappers, + Consumer> handler) { + if (ingestDocumentWrappers.isEmpty()) { handler.accept(Collections.emptyList()); return; } - int size = ingestDocuments.size(); + int size = ingestDocumentWrappers.size(); AtomicInteger counter = new AtomicInteger(size); - AtomicArray> results = new AtomicArray<>(size); + AtomicArray results = new AtomicArray<>(size); for (int i = 0; i < size; ++i) { - innerExecute(i, ingestDocuments.get(i), results, counter, handler); + innerExecute(i, ingestDocumentWrappers.get(i), results, counter, handler); } } - private void innerExecute(int slot, IngestDocument ingestDocument, - AtomicArray> results, AtomicInteger counter, - Consumer>> handler) { - execute(ingestDocument, (doc, ex) -> { - if (ex != null) { - results.set(slot, new Tuple<>(null, ex)); - } else { - results.set(slot, new Tuple<>(doc, null)); - } + private void innerExecute(int slot, IngestDocumentWrapper ingestDocumentWrapper, + AtomicArray results, AtomicInteger counter, + Consumer> handler) { + execute(ingestDocumentWrapper.getIngestDocument(), (doc, ex) -> { + results.set(slot, new IngestDocumentWrapper(ingestDocumentWrapper.getSlot(), doc, ex)); if (counter.decrementAndGet() == 0) { handler.accept(results.asList()); } From 384ba9d792942f3a999566f552bdec05dc9a8ed2 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Mon, 15 Apr 2024 18:30:41 +0800 Subject: [PATCH 03/17] Remove unnecessary change Signed-off-by: Liyun Xiu --- .../action/bulk/TransportBulkAction.java | 44 ++++++++++-- .../bulk/TransportBulkActionHelper.java | 68 ------------------- .../org/opensearch/ingest/IngestService.java | 54 ++++++++++----- .../ingest/IngestServiceHelper.java | 3 - .../bulk/TransportBulkActionHelperTests.java | 61 ----------------- .../opensearch/ingest/IngestServiceTests.java | 3 +- 6 files changed, 76 insertions(+), 157 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java delete mode 100644 server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java index a8c043f57e473..bdf1d6a369cab 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java @@ -208,6 +208,24 @@ public TransportBulkAction( this.tracer = tracer; } + /** + * Retrieves the {@link IndexRequest} from the provided {@link DocWriteRequest} for index or upsert actions. Upserts are + * modeled as {@link IndexRequest} inside the {@link UpdateRequest}. Ignores {@link org.opensearch.action.delete.DeleteRequest}'s + * + * @param docWriteRequest The request to find the {@link IndexRequest} + * @return the found {@link IndexRequest} or {@code null} if one can not be found. + */ + public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteRequest) { + IndexRequest indexRequest = null; + if (docWriteRequest instanceof IndexRequest) { + indexRequest = (IndexRequest) docWriteRequest; + } else if (docWriteRequest instanceof UpdateRequest) { + UpdateRequest updateRequest = (UpdateRequest) docWriteRequest; + indexRequest = updateRequest.docAsUpsert() ? updateRequest.doc() : updateRequest.upsertRequest(); + } + return indexRequest; + } + @Override protected void doExecute(Task task, BulkRequest bulkRequest, ActionListener listener) { final boolean isOnlySystem = isOnlySystem(bulkRequest, clusterService.state().metadata().getIndicesLookup(), systemIndices); @@ -225,12 +243,27 @@ protected void doInternalExecute(Task task, BulkRequest bulkRequest, String exec final long startTime = relativeTime(); final AtomicArray responses = new AtomicArray<>(bulkRequest.requests.size()); + boolean hasIndexRequestsWithPipelines = false; final Metadata metadata = clusterService.state().getMetadata(); final Version minNodeVersion = clusterService.state().getNodes().getMinNodeVersion(); - TransportBulkActionHelper.requestsValidation(bulkRequest.requests, minNodeVersion); + for (DocWriteRequest actionRequest : bulkRequest.requests) { + IndexRequest indexRequest = getIndexWriteRequest(actionRequest); + if (indexRequest != null) { + // Each index request needs to be evaluated, because this method also modifies the IndexRequest + boolean indexRequestHasPipeline = IngestService.resolvePipelines(actionRequest, indexRequest, metadata); + hasIndexRequestsWithPipelines |= indexRequestHasPipeline; + } + + if (actionRequest instanceof IndexRequest) { + IndexRequest ir = (IndexRequest) actionRequest; + ir.checkAutoIdWithOpTypeCreateSupportedByVersion(minNodeVersion); + if (ir.getAutoGeneratedTimestamp() != IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP) { + throw new IllegalArgumentException("autoGeneratedTimestamp should not be set externally"); + } + } + } - boolean hasIndexRequestsWithPipelines = TransportBulkActionHelper.resolvePipelines(bulkRequest.requests, metadata); if (hasIndexRequestsWithPipelines) { // this method (doExecute) will be called again, but with the bulk requests updated from the ingest node processing but // also with IngestService.NOOP_PIPELINE_NAME on each request. This ensures that this on the second time through this method, @@ -239,7 +272,7 @@ protected void doInternalExecute(Task task, BulkRequest bulkRequest, String exec if (Assertions.ENABLED) { final boolean arePipelinesResolved = bulkRequest.requests() .stream() - .map(TransportBulkActionHelper::getIndexWriteRequest) + .map(TransportBulkAction::getIndexWriteRequest) .filter(Objects::nonNull) .allMatch(IndexRequest::isPipelineResolved); assert arePipelinesResolved : bulkRequest; @@ -969,7 +1002,7 @@ ActionListener wrapActionListenerIfNeeded(long ingestTookInMillis, } synchronized void markItemAsDropped(int slot) { - IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(bulkRequest.requests().get(slot)); + IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(bulkRequest.requests().get(slot)); failedSlots.set(slot); final String id = indexRequest.id() == null ? DROPPED_ITEM_WITH_AUTO_GENERATED_ID : indexRequest.id(); itemResponses.add( @@ -989,7 +1022,8 @@ synchronized void markItemAsDropped(int slot) { } synchronized void markItemAsFailed(int slot, Exception e) { - IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(bulkRequest.requests().get(slot)); + IndexRequest indexRequest = TransportBulkAction. + getIndexWriteRequest(bulkRequest.requests().get(slot)); logger.debug( String.format( Locale.ROOT, diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java deleted file mode 100644 index 185baaac15040..0000000000000 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkActionHelper.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.action.bulk; - -import org.opensearch.Version; -import org.opensearch.action.DocWriteRequest; -import org.opensearch.action.index.IndexRequest; -import org.opensearch.action.update.UpdateRequest; -import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.ingest.IngestService; - -import java.util.List; - -/** - * A utility class for {@link TransportBulkAction}. - */ -public final class TransportBulkActionHelper { - private TransportBulkActionHelper() {} - - static void requestsValidation(final List> requests, final Version minNodeVersion) { - for (DocWriteRequest actionRequest : requests) { - if (actionRequest instanceof IndexRequest) { - IndexRequest ir = (IndexRequest) actionRequest; - ir.checkAutoIdWithOpTypeCreateSupportedByVersion(minNodeVersion); - if (ir.getAutoGeneratedTimestamp() != IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP) { - throw new IllegalArgumentException("autoGeneratedTimestamp should not be set externally"); - } - } - } - } - - /** - * Retrieves the {@link IndexRequest} from the provided {@link DocWriteRequest} for index or upsert actions. Upserts are - * modeled as {@link IndexRequest} inside the {@link UpdateRequest}. Ignores {@link org.opensearch.action.delete.DeleteRequest}'s - * - * @param docWriteRequest The request to find the {@link IndexRequest} - * @return the found {@link IndexRequest} or {@code null} if one can not be found. - */ - public static IndexRequest getIndexWriteRequest(DocWriteRequest docWriteRequest) { - IndexRequest indexRequest = null; - if (docWriteRequest instanceof IndexRequest) { - indexRequest = (IndexRequest) docWriteRequest; - } else if (docWriteRequest instanceof UpdateRequest) { - UpdateRequest updateRequest = (UpdateRequest) docWriteRequest; - indexRequest = updateRequest.docAsUpsert() ? updateRequest.doc() : updateRequest.upsertRequest(); - } - return indexRequest; - } - - static boolean resolvePipelines(final List> requests, Metadata metadata) { - boolean hasIndexRequestsWithPipelines = false; - for (DocWriteRequest actionRequest : requests) { - IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(actionRequest); - if (indexRequest != null) { - // Each index request needs to be evaluated, because this method also modifies the IndexRequest - boolean indexRequestHasPipeline = IngestService.resolvePipelines(actionRequest, indexRequest, metadata); - hasIndexRequestsWithPipelines |= indexRequestHasPipeline; - } - } - return hasIndexRequestsWithPipelines; - } -} diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index ac9e03a24a5c3..8ef4756241ee0 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -41,7 +41,7 @@ import org.opensearch.action.DocWriteRequest; import org.opensearch.action.bulk.BatchIngestionOption; import org.opensearch.action.bulk.BulkRequest; -import org.opensearch.action.bulk.TransportBulkActionHelper; +import org.opensearch.action.bulk.TransportBulkAction; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.ingest.DeletePipelineRequest; import org.opensearch.action.ingest.PutPipelineRequest; @@ -530,7 +530,7 @@ public void executeBulkRequest( BiConsumer onCompletion, IntConsumer onDropped, String executorName, - BulkRequest originalBulkRequst + BulkRequest originalBulkRequest ) { threadPool.executor(executorName).execute(new AbstractRunnable() { @@ -547,7 +547,7 @@ protected void doRun() { int i = 0; List indexRequestWrappers = new ArrayList<>(); for (DocWriteRequest actionRequest : actionRequests) { - IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(actionRequest); + IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(actionRequest); if (indexRequest == null) { if (counter.decrementAndGet() == 0) { onCompletion.accept(originalThread, null); @@ -581,15 +581,14 @@ protected void doRun() { } indexRequestWrappers.add(new IndexRequestWrapper(i, indexRequest, pipelines, hasFinalPipeline)); - i++; } - BatchIngestionOption batchOption = originalBulkRequst.batchIngestionOption(); - int batchSize = originalBulkRequst.maximumBatchSize(); + BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); + int batchSize = originalBulkRequest.maximumBatchSize(); if (shouldExecutePipelineInBatch(batchOption, indexRequestWrappers.size(), batchSize)) { - List> batches = handleBatch(batchSize, indexRequestWrappers); - logger.info("batchSize: {}, batches: {}", batchSize, batches.size()); + List> batches = prepareBatches(batchSize, indexRequestWrappers); + logger.debug("batchSize: {}, batches: {}", batchSize, batches.size()); for (List batch : batches) { executePipelinesInBatchRequests(batch.stream() .map(IndexRequestWrapper::getSlot) @@ -621,19 +620,37 @@ private boolean shouldExecutePipelineInBatch(BatchIngestionOption batchOption, i return batchOption == BatchIngestionOption.ENABLED && documentSize > 1 && batchSize > 1; } - private List> handleBatch(int batchSize, List indexRequestWrappers) { - final Map> indexRequestsPerIndexMap = new HashMap<>(); + /** + * IndexRequests are grouped by unique (index + pipeline_ids) before batching. + * Only IndexRequests in the same group could be batched. It's to ensure batched documents always + * flow through the same pipeline together. + * + * An IndexRequest could be preprocessed by at most two pipelines: default_pipeline and final_pipeline. + * A final_pipeline is configured on index level. The default_pipeline for a IndexRequest in a _bulk API + * could come from three places: + * 1. bound with index + * 2. a request parameter of _bulk API + * 3. a parameter of an IndexRequest. + */ + private List> prepareBatches(int batchSize, List indexRequestWrappers) { + final Map> indexRequestsPerIndexAndPipelines = new HashMap<>(); for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { + // IndexRequests are grouped by their index + pipeline ids + List indexAndPipelineIds = new ArrayList<>(); String index = indexRequestWrapper.getIndexRequest().index(); - indexRequestsPerIndexMap.putIfAbsent(index, new ArrayList<>()); - indexRequestsPerIndexMap.get(index).add(indexRequestWrapper); + List pipelines = indexRequestWrapper.getPipelines(); + indexAndPipelineIds.add(index); + indexAndPipelineIds.addAll(pipelines); + int hashCode = indexAndPipelineIds.hashCode(); + indexRequestsPerIndexAndPipelines.putIfAbsent(hashCode, new ArrayList<>()); + indexRequestsPerIndexAndPipelines.get(hashCode).add(indexRequestWrapper); } List> batchedIndexRequests = new ArrayList<>(); - for (Map.Entry> indexRequestsPerIndex : indexRequestsPerIndexMap.entrySet()) { - for (int i = 0; i < indexRequestsPerIndex.getValue().size(); i += batchSize) { + for (Map.Entry> indexRequestsPerKey : indexRequestsPerIndexAndPipelines.entrySet()) { + for (int i = 0; i < indexRequestsPerKey.getValue().size(); i += batchSize) { batchedIndexRequests.add(new ArrayList<>( - indexRequestsPerIndex.getValue().subList(i, - i + Math.min(batchSize, indexRequestsPerIndex.getValue().size() - i)))); + indexRequestsPerKey.getValue().subList(i, + i + Math.min(batchSize, indexRequestsPerKey.getValue().size() - i)))); } } return batchedIndexRequests; @@ -695,7 +712,7 @@ private void executePipelinesInBatchRequests( String originalIndex = indexRequests.get(0).indices()[0]; Map slotIndexRequestMap = IngestServiceHelper.createSlotIndexRequestMap(slots, indexRequests); - innerExecute(slots, indexRequests, pipeline, onDropped, results -> { + innerBatchExecute(slots, indexRequests, pipeline, onDropped, results -> { for (int i = 0; i < results.size(); ++i) { if (results.get(i).getException() != null) { IndexRequest indexRequest = slotIndexRequestMap.get(results.get(i).getSlot()); @@ -980,7 +997,7 @@ private void innerExecute( }); } - private void innerExecute( + private void innerBatchExecute( List slots, List indexRequests, Pipeline pipeline, @@ -1006,6 +1023,7 @@ private void innerExecute( AtomicInteger counter = new AtomicInteger(size); List allResults = Collections.synchronizedList(new ArrayList<>()); pipeline.batchExecute(ingestDocumentWrappers, results -> { + if (results.isEmpty()) return; allResults.addAll(results); if (counter.addAndGet(-results.size()) == 0) { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos); diff --git a/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java b/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java index 482a821feb288..724a0b6ee9bba 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java +++ b/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java @@ -22,9 +22,6 @@ public class IngestServiceHelper { private IngestServiceHelper() {} public static void updateIndexRequestWithIngestDocument(IndexRequest indexRequest, IngestDocument ingestDocument) { - if (ingestDocument == null || indexRequest == null) { - return; - } Map metadataMap = ingestDocument.extractMetadata(); // it's fine to set all metadata fields all the time, as ingest document holds their starting values // before ingestion, which might also get modified during ingestion. diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java deleted file mode 100644 index f4953c3377bf4..0000000000000 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionHelperTests.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -package org.opensearch.action.bulk; - -import org.opensearch.action.delete.DeleteRequest; -import org.opensearch.action.index.IndexRequest; -import org.opensearch.action.update.UpdateRequest; -import org.opensearch.test.OpenSearchTestCase; - -import static java.util.Collections.emptyMap; - -public class TransportBulkActionHelperTests extends OpenSearchTestCase { - - public void testGetIndexWriteRequest() { - IndexRequest indexRequest = new IndexRequest("index").id("id1").source(emptyMap()); - UpdateRequest upsertRequest = new UpdateRequest("index", "id1").upsert(indexRequest).script(mockScript("1")); - UpdateRequest docAsUpsertRequest = new UpdateRequest("index", "id2").doc(indexRequest).docAsUpsert(true); - UpdateRequest scriptedUpsert = new UpdateRequest("index", "id2").upsert(indexRequest).script(mockScript("1")).scriptedUpsert(true); - - assertEquals(TransportBulkActionHelper.getIndexWriteRequest(indexRequest), indexRequest); - assertEquals(TransportBulkActionHelper.getIndexWriteRequest(upsertRequest), indexRequest); - assertEquals(TransportBulkActionHelper.getIndexWriteRequest(docAsUpsertRequest), indexRequest); - assertEquals(TransportBulkActionHelper.getIndexWriteRequest(scriptedUpsert), indexRequest); - - DeleteRequest deleteRequest = new DeleteRequest("index", "id"); - assertNull(TransportBulkActionHelper.getIndexWriteRequest(deleteRequest)); - - UpdateRequest badUpsertRequest = new UpdateRequest("index", "id1"); - assertNull(TransportBulkActionHelper.getIndexWriteRequest(badUpsertRequest)); - } -} diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 72ac5e8a16733..23195eda525df 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -40,7 +40,6 @@ import org.opensearch.action.DocWriteRequest; import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.TransportBulkAction; -import org.opensearch.action.bulk.TransportBulkActionHelper; import org.opensearch.action.delete.DeleteRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.ingest.DeletePipelineRequest; @@ -1212,7 +1211,7 @@ public void testBulkRequestExecution() throws Exception { verify(requestItemErrorHandler, never()).accept(any(), any()); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); for (DocWriteRequest docWriteRequest : bulkRequest.requests()) { - IndexRequest indexRequest = TransportBulkActionHelper.getIndexWriteRequest(docWriteRequest); + IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(docWriteRequest); assertThat(indexRequest, notNullValue()); assertThat(indexRequest.getContentType(), equalTo(xContentType)); } From 21f0d71f99e005702472cd92c20d73a0d49a955b Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Mon, 15 Apr 2024 18:42:11 +0800 Subject: [PATCH 04/17] Revert some unnecessary test change Signed-off-by: Liyun Xiu --- .../opensearch/action/bulk/BulkRequest.java | 6 +++++- .../action/bulk/TransportBulkAction.java | 7 ++----- .../action/bulk/TransportBulkActionTests.java | 18 ++++++++++++++++++ 3 files changed, 25 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 76932b2396a9e..a1931f5ee85af 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -372,7 +372,11 @@ public final BatchIngestionOption batchIngestionOption() { * @return {@link BulkRequest} */ public final BulkRequest maximumBatchSize(int size) { - this.maximumBatchSize = size; + if (size > 1) { + this.maximumBatchSize = size; + } else { + this.maximumBatchSize = 1; + } return this; } diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java index bdf1d6a369cab..19ffb12859183 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java @@ -246,7 +246,6 @@ protected void doInternalExecute(Task task, BulkRequest bulkRequest, String exec boolean hasIndexRequestsWithPipelines = false; final Metadata metadata = clusterService.state().getMetadata(); final Version minNodeVersion = clusterService.state().getNodes().getMinNodeVersion(); - for (DocWriteRequest actionRequest : bulkRequest.requests) { IndexRequest indexRequest = getIndexWriteRequest(actionRequest); if (indexRequest != null) { @@ -1002,7 +1001,7 @@ ActionListener wrapActionListenerIfNeeded(long ingestTookInMillis, } synchronized void markItemAsDropped(int slot) { - IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(bulkRequest.requests().get(slot)); + IndexRequest indexRequest = getIndexWriteRequest(bulkRequest.requests().get(slot)); failedSlots.set(slot); final String id = indexRequest.id() == null ? DROPPED_ITEM_WITH_AUTO_GENERATED_ID : indexRequest.id(); itemResponses.add( @@ -1022,8 +1021,7 @@ synchronized void markItemAsDropped(int slot) { } synchronized void markItemAsFailed(int slot, Exception e) { - IndexRequest indexRequest = TransportBulkAction. - getIndexWriteRequest(bulkRequest.requests().get(slot)); + IndexRequest indexRequest = getIndexWriteRequest(bulkRequest.requests().get(slot)); logger.debug( String.format( Locale.ROOT, @@ -1045,5 +1043,4 @@ synchronized void markItemAsFailed(int slot, Exception e) { } } - } diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java index 93a7a0d55aeb9..6bbd740df7f9c 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTests.java @@ -207,6 +207,24 @@ public void testDeleteNonExistingDocExternalGteVersionCreatesIndex() throws Exce assertTrue(bulkAction.indexCreated); } + public void testGetIndexWriteRequest() throws Exception { + IndexRequest indexRequest = new IndexRequest("index").id("id1").source(emptyMap()); + UpdateRequest upsertRequest = new UpdateRequest("index", "id1").upsert(indexRequest).script(mockScript("1")); + UpdateRequest docAsUpsertRequest = new UpdateRequest("index", "id2").doc(indexRequest).docAsUpsert(true); + UpdateRequest scriptedUpsert = new UpdateRequest("index", "id2").upsert(indexRequest).script(mockScript("1")).scriptedUpsert(true); + + assertEquals(TransportBulkAction.getIndexWriteRequest(indexRequest), indexRequest); + assertEquals(TransportBulkAction.getIndexWriteRequest(upsertRequest), indexRequest); + assertEquals(TransportBulkAction.getIndexWriteRequest(docAsUpsertRequest), indexRequest); + assertEquals(TransportBulkAction.getIndexWriteRequest(scriptedUpsert), indexRequest); + + DeleteRequest deleteRequest = new DeleteRequest("index", "id"); + assertNull(TransportBulkAction.getIndexWriteRequest(deleteRequest)); + + UpdateRequest badUpsertRequest = new UpdateRequest("index", "id1"); + assertNull(TransportBulkAction.getIndexWriteRequest(badUpsertRequest)); + } + public void testProhibitAppendWritesInBackingIndices() throws Exception { String dataStreamName = "logs-foobar"; ClusterState clusterState = createDataStream(dataStreamName); From 4ba55c2531fbcc2c154ff35bd95527cf94fefa58 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 16 Apr 2024 14:05:50 +0800 Subject: [PATCH 05/17] Keep executeBulkRequest main logic untouched Signed-off-by: Liyun Xiu --- .../org/opensearch/ingest/IngestService.java | 183 +++++++++++++----- .../ingest/IngestServiceHelper.java | 65 ------- 2 files changed, 130 insertions(+), 118 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 8ef4756241ee0..98296f0392979 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -508,21 +508,6 @@ void validatePipeline(Map ingestInfos, PutPipelineReq ExceptionsHelper.rethrowAndSuppress(exceptions); } - /** - * Only used for testing - */ - public void executeBulkRequest( - int numberOfActionRequests, - Iterable> actionRequests, - BiConsumer onFailure, - BiConsumer onCompletion, - IntConsumer onDropped, - String executorName - ) { - executeBulkRequest(numberOfActionRequests, actionRequests, onFailure, onCompletion, - onDropped, executorName, new BulkRequest()); - } - public void executeBulkRequest( int numberOfActionRequests, Iterable> actionRequests, @@ -541,11 +526,17 @@ public void onFailure(Exception e) { @Override protected void doRun() { + BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); + int batchSize = originalBulkRequest.maximumBatchSize(); + if (shouldExecuteBulkRequestInBatch(batchOption, originalBulkRequest.requests().size(), batchSize)) { + runBulkRequestInBatch(numberOfActionRequests, actionRequests, onFailure, onCompletion, onDropped, + originalBulkRequest); + return; + } + final Thread originalThread = Thread.currentThread(); final AtomicInteger counter = new AtomicInteger(numberOfActionRequests); - int i = 0; - List indexRequestWrappers = new ArrayList<>(); for (DocWriteRequest actionRequest : actionRequests) { IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(actionRequest); if (indexRequest == null) { @@ -556,7 +547,6 @@ protected void doRun() { i++; continue; } - final String pipelineId = indexRequest.getPipeline(); indexRequest.setPipeline(NOOP_PIPELINE_NAME); final String finalPipelineId = indexRequest.getFinalPipeline(); @@ -580,43 +570,94 @@ protected void doRun() { continue; } - indexRequestWrappers.add(new IndexRequestWrapper(i, indexRequest, pipelines, hasFinalPipeline)); + executePipelines( + i, + pipelines.iterator(), + hasFinalPipeline, + indexRequest, + onDropped, + onFailure, + counter, + onCompletion, + originalThread + ); i++; } + } + }); + } - BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); - int batchSize = originalBulkRequest.maximumBatchSize(); - if (shouldExecutePipelineInBatch(batchOption, indexRequestWrappers.size(), batchSize)) { - List> batches = prepareBatches(batchSize, indexRequestWrappers); - logger.debug("batchSize: {}, batches: {}", batchSize, batches.size()); - for (List batch : batches) { - executePipelinesInBatchRequests(batch.stream() - .map(IndexRequestWrapper::getSlot) - .collect(Collectors.toList()), - batch.get(0).getPipelines().iterator(), - batch.get(0).isHasFinalPipeline(), - batch.stream().map(IndexRequestWrapper::getIndexRequest).collect(Collectors.toList()), - onDropped, - onFailure, - counter, - onCompletion, - originalThread - ); - } - } else { - for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { - executePipelines(indexRequestWrapper.getSlot(), - indexRequestWrapper.getPipelines().iterator(), - indexRequestWrapper.isHasFinalPipeline(), - indexRequestWrapper.getIndexRequest(), - onDropped, onFailure, counter, onCompletion, originalThread); - } + private void runBulkRequestInBatch(int numberOfActionRequests, + Iterable> actionRequests, + BiConsumer onFailure, + BiConsumer onCompletion, + IntConsumer onDropped, + BulkRequest originalBulkRequest) { + + final Thread originalThread = Thread.currentThread(); + final AtomicInteger counter = new AtomicInteger(numberOfActionRequests); + int i = 0; + List indexRequestWrappers = new ArrayList<>(); + for (DocWriteRequest actionRequest : actionRequests) { + IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(actionRequest); + if (indexRequest == null) { + if (counter.decrementAndGet() == 0) { + onCompletion.accept(originalThread, null); } + assert counter.get() >= 0; + i++; + continue; } - }); + + final String pipelineId = indexRequest.getPipeline(); + indexRequest.setPipeline(NOOP_PIPELINE_NAME); + final String finalPipelineId = indexRequest.getFinalPipeline(); + indexRequest.setFinalPipeline(NOOP_PIPELINE_NAME); + boolean hasFinalPipeline = true; + final List pipelines; + if (IngestService.NOOP_PIPELINE_NAME.equals(pipelineId) == false + && IngestService.NOOP_PIPELINE_NAME.equals(finalPipelineId) == false) { + pipelines = Arrays.asList(pipelineId, finalPipelineId); + } else if (IngestService.NOOP_PIPELINE_NAME.equals(pipelineId) == false) { + pipelines = Collections.singletonList(pipelineId); + hasFinalPipeline = false; + } else if (IngestService.NOOP_PIPELINE_NAME.equals(finalPipelineId) == false) { + pipelines = Collections.singletonList(finalPipelineId); + } else { + if (counter.decrementAndGet() == 0) { + onCompletion.accept(originalThread, null); + } + assert counter.get() >= 0; + i++; + continue; + } + + indexRequestWrappers.add(new IndexRequestWrapper(i, indexRequest, pipelines, hasFinalPipeline)); + i++; + } + + BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); + int batchSize = originalBulkRequest.maximumBatchSize(); + List> batches = prepareBatches(batchSize, indexRequestWrappers); + logger.debug("batchSize: {}, batches: {}", batchSize, batches.size()); + + for (List batch : batches) { + executePipelinesInBatchRequests(batch.stream() + .map(IndexRequestWrapper::getSlot) + .collect(Collectors.toList()), + batch.get(0).getPipelines().iterator(), + batch.get(0).isHasFinalPipeline(), + batch.stream().map(IndexRequestWrapper::getIndexRequest).collect(Collectors.toList()), + onDropped, + onFailure, + counter, + onCompletion, + originalThread + ); + } } - private boolean shouldExecutePipelineInBatch(BatchIngestionOption batchOption, int documentSize, int batchSize) { + private boolean shouldExecuteBulkRequestInBatch(BatchIngestionOption batchOption, int documentSize, int batchSize) { return batchOption == BatchIngestionOption.ENABLED && documentSize > 1 && batchSize > 1; } @@ -710,8 +751,7 @@ private void executePipelinesInBatchRequests( } Pipeline pipeline = holder.pipeline; String originalIndex = indexRequests.get(0).indices()[0]; - Map slotIndexRequestMap = IngestServiceHelper.createSlotIndexRequestMap(slots, - indexRequests); + Map slotIndexRequestMap = createSlotIndexRequestMap(slots, indexRequests); innerBatchExecute(slots, indexRequests, pipeline, onDropped, results -> { for (int i = 0; i < results.size(); ++i) { if (results.get(i).getException() != null) { @@ -991,7 +1031,7 @@ private void innerExecute( itemDroppedHandler.accept(slot); handler.accept(null); } else { - IngestServiceHelper.updateIndexRequestWithIngestDocument(indexRequest, ingestDocument); + updateIndexRequestWithIngestDocument(indexRequest, ingestDocument); handler.accept(null); } }); @@ -1018,7 +1058,7 @@ private void innerBatchExecute( Map slotToindexRequestMap = new HashMap<>(); for (int i = 0; i < slots.size(); ++i) { slotToindexRequestMap.put(slots.get(i), indexRequests.get(i)); - ingestDocumentWrappers.add(IngestServiceHelper.toIngestDocumentWrapper(slots.get(i), indexRequests.get(i))); + ingestDocumentWrappers.add(toIngestDocumentWrapper(slots.get(i), indexRequests.get(i))); } AtomicInteger counter = new AtomicInteger(size); List allResults = Collections.synchronizedList(new ArrayList<>()); @@ -1046,7 +1086,7 @@ private void innerBatchExecute( dropped.forEach(t -> itemDroppedHandler.accept(t.getSlot())); } else { for (IngestDocumentWrapper ingestDocumentWrapper : succeeded) { - IngestServiceHelper.updateIndexRequestWithIngestDocument( + updateIndexRequestWithIngestDocument( slotToindexRequestMap.get(ingestDocumentWrapper.getSlot()), ingestDocumentWrapper.getIngestDocument()); } @@ -1242,4 +1282,41 @@ static class PipelineHolder { } } + public static void updateIndexRequestWithIngestDocument(IndexRequest indexRequest, IngestDocument ingestDocument) { + Map metadataMap = ingestDocument.extractMetadata(); + // it's fine to set all metadata fields all the time, as ingest document holds their starting values + // before ingestion, which might also get modified during ingestion. + indexRequest.index((String) metadataMap.get(IngestDocument.Metadata.INDEX)); + indexRequest.id((String) metadataMap.get(IngestDocument.Metadata.ID)); + indexRequest.routing((String) metadataMap.get(IngestDocument.Metadata.ROUTING)); + indexRequest.version(((Number) metadataMap.get(IngestDocument.Metadata.VERSION)).longValue()); + if (metadataMap.get(IngestDocument.Metadata.VERSION_TYPE) != null) { + indexRequest.versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.Metadata.VERSION_TYPE))); + } + if (metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO) != null) { + indexRequest.setIfSeqNo(((Number) metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO)).longValue()); + } + if (metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM) != null) { + indexRequest.setIfPrimaryTerm(((Number) metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM)).longValue()); + } + indexRequest.source(ingestDocument.getSourceAndMetadata(), indexRequest.getContentType()); + } + + private static IngestDocument toIngestDocument(IndexRequest indexRequest) { + return new IngestDocument(indexRequest.index(), indexRequest.id(), indexRequest.routing(), + indexRequest.version(), indexRequest.versionType(), indexRequest.sourceAsMap()); + } + + private static IngestDocumentWrapper toIngestDocumentWrapper(int slot, IndexRequest indexRequest) { + return new IngestDocumentWrapper(slot, toIngestDocument(indexRequest), null); + } + + private static Map createSlotIndexRequestMap(List slots, + List indexRequests) { + Map slotIndexRequestMap = new HashMap<>(); + for (int i = 0; i < slots.size(); ++i) { + slotIndexRequestMap.put(slots.get(i), indexRequests.get(i)); + } + return slotIndexRequestMap; + } } diff --git a/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java b/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java deleted file mode 100644 index 724a0b6ee9bba..0000000000000 --- a/server/src/main/java/org/opensearch/ingest/IngestServiceHelper.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.ingest; - -import org.opensearch.action.index.IndexRequest; -import org.opensearch.index.VersionType; - -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * This is a helper class for static functions which can live outside IngestService. - */ -public class IngestServiceHelper { - private IngestServiceHelper() {} - - public static void updateIndexRequestWithIngestDocument(IndexRequest indexRequest, IngestDocument ingestDocument) { - Map metadataMap = ingestDocument.extractMetadata(); - // it's fine to set all metadata fields all the time, as ingest document holds their starting values - // before ingestion, which might also get modified during ingestion. - indexRequest.index((String) metadataMap.get(IngestDocument.Metadata.INDEX)); - indexRequest.id((String) metadataMap.get(IngestDocument.Metadata.ID)); - indexRequest.routing((String) metadataMap.get(IngestDocument.Metadata.ROUTING)); - indexRequest.version(((Number) metadataMap.get(IngestDocument.Metadata.VERSION)).longValue()); - if (metadataMap.get(IngestDocument.Metadata.VERSION_TYPE) != null) { - indexRequest.versionType(VersionType.fromString((String) metadataMap.get(IngestDocument.Metadata.VERSION_TYPE))); - } - if (metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO) != null) { - indexRequest.setIfSeqNo(((Number) metadataMap.get(IngestDocument.Metadata.IF_SEQ_NO)).longValue()); - } - if (metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM) != null) { - indexRequest.setIfPrimaryTerm(((Number) metadataMap.get(IngestDocument.Metadata.IF_PRIMARY_TERM)).longValue()); - } - indexRequest.source(ingestDocument.getSourceAndMetadata(), indexRequest.getContentType()); - } - - - public static IngestDocument toIngestDocument(IndexRequest indexRequest) { - return new IngestDocument(indexRequest.index(), indexRequest.id(), indexRequest.routing(), - indexRequest.version(), indexRequest.versionType(), indexRequest.sourceAsMap()); - } - - public static IngestDocumentWrapper toIngestDocumentWrapper(int slot, IndexRequest indexRequest) { - return new IngestDocumentWrapper(slot, toIngestDocument(indexRequest), null); - } - - public static Map createSlotIndexRequestMap(List slots, - List indexRequests) { - Map slotIndexRequestMap = new HashMap<>(); - for (int i = 0; i < slots.size(); ++i) { - slotIndexRequestMap.put(slots.get(i), indexRequests.get(i)); - } - return slotIndexRequestMap; - } -} - - - From d9a832d4f683cc1df4e138ea4a908af87a881606 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Thu, 18 Apr 2024 18:30:41 +0800 Subject: [PATCH 06/17] Add UT Signed-off-by: Liyun Xiu --- .../opensearch/ingest/CompoundProcessor.java | 1 + .../org/opensearch/ingest/IngestService.java | 8 +- .../bulk/TransportBulkActionIngestTests.java | 28 +-- .../ingest/CompoundProcessorTests.java | 162 ++++++++++++++++++ .../ingest/IngestDocumentPreparer.java | 32 ++++ .../ingest/IngestDocumentWrapperTest.java | 46 +++++ .../opensearch/ingest/IngestServiceTests.java | 149 ++++++++++++++-- .../org/opensearch/ingest/ProcessorTests.java | 70 ++++++++ 8 files changed, 466 insertions(+), 30 deletions(-) create mode 100644 server/src/test/java/org/opensearch/ingest/IngestDocumentPreparer.java create mode 100644 server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTest.java create mode 100644 server/src/test/java/org/opensearch/ingest/ProcessorTests.java diff --git a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java index b51189acc3360..e0a5eb3597382 100644 --- a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java @@ -177,6 +177,7 @@ public void innerBatchExecute(int currentProcessor, List Map slotToWrapperMap = createSlotIngestDocumentWrapperMap(ingestDocumentWrappers); processor.batchExecute(ingestDocumentWrappers, results -> { + if (results.isEmpty()) return; allResults.addAll(results); if (counter.addAndGet(-results.size()) == 0) { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 98296f0392979..5f80c2cd20971 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -636,7 +636,6 @@ private void runBulkRequestInBatch(int numberOfActionRequests, i++; } - BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); int batchSize = originalBulkRequest.maximumBatchSize(); List> batches = prepareBatches(batchSize, indexRequestWrappers); logger.debug("batchSize: {}, batches: {}", batchSize, batches.size()); @@ -673,7 +672,7 @@ private boolean shouldExecuteBulkRequestInBatch(BatchIngestionOption batchOption * 2. a request parameter of _bulk API * 3. a parameter of an IndexRequest. */ - private List> prepareBatches(int batchSize, List indexRequestWrappers) { + static List> prepareBatches(int batchSize, List indexRequestWrappers) { final Map> indexRequestsPerIndexAndPipelines = new HashMap<>(); for (IndexRequestWrapper indexRequestWrapper : indexRequestWrappers) { // IndexRequests are grouped by their index + pipeline ids @@ -697,13 +696,14 @@ private List> prepareBatches(int batchSize, List pipelines; private final boolean hasFinalPipeline; - private IndexRequestWrapper(int slot, IndexRequest indexRequest, List pipelines, boolean hasFinalPipeline) { + IndexRequestWrapper(int slot, IndexRequest indexRequest, List pipelines, boolean hasFinalPipeline) { this.slot = slot; this.indexRequest = indexRequest; this.pipelines = pipelines; diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java index 141c630b94020..65f96f12a571b 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java @@ -341,7 +341,8 @@ public void testIngestLocal() throws Exception { failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.WRITE) + eq(Names.WRITE), + bulkRequest ); completionHandler.getValue().accept(null, exception); assertTrue(failureCalled.get()); @@ -378,7 +379,8 @@ public void testSingleItemBulkActionIngestLocal() throws Exception { failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.WRITE) + eq(Names.WRITE), + any() ); completionHandler.getValue().accept(null, exception); assertTrue(failureCalled.get()); @@ -424,7 +426,8 @@ public void testIngestSystemLocal() throws Exception { failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.SYSTEM_WRITE) + eq(Names.SYSTEM_WRITE), + eq(bulkRequest) ); completionHandler.getValue().accept(null, exception); assertTrue(failureCalled.get()); @@ -455,7 +458,7 @@ public void testIngestForward() throws Exception { action.execute(null, bulkRequest, listener); // should not have executed ingest locally - verify(ingestService, never()).executeBulkRequest(anyInt(), any(), any(), any(), any(), any()); + verify(ingestService, never()).executeBulkRequest(anyInt(), any(), any(), any(), any(), any(), any()); // but instead should have sent to a remote node with the transport service ArgumentCaptor node = ArgumentCaptor.forClass(DiscoveryNode.class); verify(transportService).sendRequest(node.capture(), eq(BulkAction.NAME), any(), remoteResponseHandler.capture()); @@ -495,7 +498,7 @@ public void testSingleItemBulkActionIngestForward() throws Exception { singleItemBulkWriteAction.execute(null, indexRequest, listener); // should not have executed ingest locally - verify(ingestService, never()).executeBulkRequest(anyInt(), any(), any(), any(), any(), any()); + verify(ingestService, never()).executeBulkRequest(anyInt(), any(), any(), any(), any(), any(), any()); // but instead should have sent to a remote node with the transport service ArgumentCaptor node = ArgumentCaptor.forClass(DiscoveryNode.class); verify(transportService).sendRequest(node.capture(), eq(BulkAction.NAME), any(), remoteResponseHandler.capture()); @@ -581,7 +584,8 @@ private void validatePipelineWithBulkUpsert(@Nullable String indexRequestIndexNa failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.WRITE) + eq(Names.WRITE), + eq(bulkRequest) ); assertEquals(indexRequest1.getPipeline(), "default_pipeline"); assertEquals(indexRequest2.getPipeline(), "default_pipeline"); @@ -624,7 +628,8 @@ public void testDoExecuteCalledTwiceCorrectly() throws Exception { failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.WRITE) + eq(Names.WRITE), + any() ); completionHandler.getValue().accept(null, exception); assertFalse(action.indexCreated); // still no index yet, the ingest node failed. @@ -711,7 +716,8 @@ public void testFindDefaultPipelineFromTemplateMatch() { failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.WRITE) + eq(Names.WRITE), + any() ); } @@ -750,7 +756,8 @@ public void testFindDefaultPipelineFromV2TemplateMatch() { failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.WRITE) + eq(Names.WRITE), + any() ); } @@ -775,7 +782,8 @@ private void validateDefaultPipeline(IndexRequest indexRequest) { failureHandler.capture(), completionHandler.capture(), any(), - eq(Names.WRITE) + eq(Names.WRITE), + any() ); assertEquals(indexRequest.getPipeline(), "default_pipeline"); completionHandler.getValue().accept(null, exception); diff --git a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java index 76301acac0c19..5d39be2665a4d 100644 --- a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java @@ -37,11 +37,17 @@ import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.LongSupplier; @@ -57,6 +63,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.ingest.IngestDocumentPreparer.SHOULD_FAIL_KEY; public class CompoundProcessorTests extends OpenSearchTestCase { private IngestDocument ingestDocument; @@ -429,6 +436,161 @@ public String getType() { assertThat(ingestProcessorException.getHeader("pipeline_origin"), equalTo(Arrays.asList("2", "1"))); } + public void testBatchExecute_happyCase() { + List wrapperList = Arrays.asList( + IngestDocumentPreparer.createIngestDocumentWrapper(1), + IngestDocumentPreparer.createIngestDocumentWrapper(2), + IngestDocumentPreparer.createIngestDocumentWrapper(3)); + TestProcessor firstProcessor = new TestProcessor(doc -> {}); + TestProcessor secondProcessor = new TestProcessor(doc -> {}); + LongSupplier relativeTimeProvider = mock(LongSupplier.class); + CompoundProcessor compoundProcessor = new CompoundProcessor( + false, + Arrays.asList(firstProcessor, secondProcessor), + null, + relativeTimeProvider + ); + + compoundProcessor.batchExecute(wrapperList, results -> { + assertEquals(firstProcessor.getInvokedCounter(), wrapperList.size()); + assertEquals(secondProcessor.getInvokedCounter(), wrapperList.size()); + assertEquals(results.size(), wrapperList.size()); + OperationStats stats = compoundProcessor.getProcessorsWithMetrics().get(0).v2().createStats(); + assertEquals(0, stats.getCurrent()); + assertEquals(3, stats.getCount()); + for (int i = 0; i < wrapperList.size(); ++i) { + assertEquals(wrapperList.get(i).getSlot(), results.get(i).getSlot()); + assertEquals(wrapperList.get(i).getIngestDocument(), results.get(i).getIngestDocument()); + assertEquals(wrapperList.get(i).getException(), results.get(i).getException()); + } + }); + } + + public void testBatchExecute_ignoreFailure() { + List wrapperList = Arrays.asList( + IngestDocumentPreparer.createIngestDocumentWrapper(1), + IngestDocumentPreparer.createIngestDocumentWrapper(2, true), + IngestDocumentPreparer.createIngestDocumentWrapper(3, true)); + TestProcessor firstProcessor = new TestProcessor(doc -> { + if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { + throw new RuntimeException("fail"); + } + }); + TestProcessor secondProcessor = new TestProcessor(doc -> {}); + TestProcessor onFailureProcessor = new TestProcessor("id2", "on_failure", null, doc -> {}); + LongSupplier relativeTimeProvider = mock(LongSupplier.class); + CompoundProcessor compoundProcessor = new CompoundProcessor( + true, + Arrays.asList(firstProcessor, secondProcessor), + singletonList(onFailureProcessor), + relativeTimeProvider + ); + + compoundProcessor.batchExecute(wrapperList, results -> { + assertEquals(firstProcessor.getInvokedCounter(), wrapperList.size()); + assertEquals(secondProcessor.getInvokedCounter(), wrapperList.size()); + assertEquals(0, onFailureProcessor.getInvokedCounter()); + assertEquals(results.size(), wrapperList.size()); + OperationStats stats = compoundProcessor.getProcessorsWithMetrics().get(0).v2().createStats(); + assertEquals(0, stats.getCurrent()); + assertEquals(3, stats.getCount()); + for (int i = 0; i < wrapperList.size(); ++i) { + assertEquals(wrapperList.get(i).getSlot(), results.get(i).getSlot()); + assertEquals(wrapperList.get(i).getIngestDocument(), results.get(i).getIngestDocument()); + assertEquals(wrapperList.get(i).getException(), results.get(i).getException()); + } + }); + } + + public void testBatchExecute_exception_no_onFailureProcessor() { + Set failureSlot = new HashSet<>(Arrays.asList(2, 3)); + List wrapperList = Arrays.asList( + IngestDocumentPreparer.createIngestDocumentWrapper(1), + IngestDocumentPreparer.createIngestDocumentWrapper(2, true), + IngestDocumentPreparer.createIngestDocumentWrapper(3, true)); + TestProcessor firstProcessor = new TestProcessor(doc -> { + if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { + throw new RuntimeException("fail"); + } + }); + TestProcessor secondProcessor = new TestProcessor(doc -> {}); + LongSupplier relativeTimeProvider = mock(LongSupplier.class); + CompoundProcessor compoundProcessor = new CompoundProcessor( + false, + Arrays.asList(firstProcessor, secondProcessor), + Collections.emptyList(), + relativeTimeProvider + ); + + AtomicInteger callCounter = new AtomicInteger(); + List totalResults = Collections.synchronizedList(new ArrayList<>()); + compoundProcessor.batchExecute(wrapperList, results -> { + totalResults.addAll(results); + if (callCounter.incrementAndGet() == 3) { + assertEquals(wrapperList.size(), firstProcessor.getInvokedCounter()); + assertEquals(1, secondProcessor.getInvokedCounter()); + assertEquals(totalResults.size(), wrapperList.size()); + OperationStats stats = compoundProcessor.getProcessorsWithMetrics().get(0).v2().createStats(); + assertEquals(0, stats.getCurrent()); + assertEquals(3, stats.getCount()); + assertEquals(2, stats.getFailedCount()); + totalResults.sort(Comparator.comparingInt(IngestDocumentWrapper::getSlot)); + for (int i = 0; i < wrapperList.size(); ++i) { + assertEquals(wrapperList.get(i).getSlot(), totalResults.get(i).getSlot()); + if (failureSlot.contains(wrapperList.get(i).getSlot())) { + assertNotNull(totalResults.get(i).getException()); + } else { + assertEquals(wrapperList.get(i).getIngestDocument(), totalResults.get(i).getIngestDocument()); + assertEquals(wrapperList.get(i).getException(), totalResults.get(i).getException()); + } + } + } + }); + } + + public void testBatchExecute_exception_with_onFailureProcessor() { + List wrapperList = Arrays.asList( + IngestDocumentPreparer.createIngestDocumentWrapper(1), + IngestDocumentPreparer.createIngestDocumentWrapper(2, true), + IngestDocumentPreparer.createIngestDocumentWrapper(3, true)); + TestProcessor firstProcessor = new TestProcessor(doc -> { + if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { + throw new RuntimeException("fail"); + } + }); + TestProcessor secondProcessor = new TestProcessor(doc -> {}); + TestProcessor onFailureProcessor = new TestProcessor("id2", "on_failure", null, doc -> {}); + LongSupplier relativeTimeProvider = mock(LongSupplier.class); + CompoundProcessor compoundProcessor = new CompoundProcessor( + false, + Arrays.asList(firstProcessor, secondProcessor), + singletonList(onFailureProcessor), + relativeTimeProvider + ); + + AtomicInteger callCounter = new AtomicInteger(); + List totalResults = Collections.synchronizedList(new ArrayList<>()); + compoundProcessor.batchExecute(wrapperList, results -> { + totalResults.addAll(results); + if (callCounter.incrementAndGet() == 3) { + assertEquals(wrapperList.size(), firstProcessor.getInvokedCounter()); + assertEquals(1, secondProcessor.getInvokedCounter()); + assertEquals(2, onFailureProcessor.getInvokedCounter()); + assertEquals(totalResults.size(), wrapperList.size()); + OperationStats stats = compoundProcessor.getProcessorsWithMetrics().get(0).v2().createStats(); + assertEquals(0, stats.getCurrent()); + assertEquals(3, stats.getCount()); + assertEquals(2, stats.getFailedCount()); + totalResults.sort(Comparator.comparingInt(IngestDocumentWrapper::getSlot)); + for (int i = 0; i < wrapperList.size(); ++i) { + assertEquals(wrapperList.get(i).getSlot(), totalResults.get(i).getSlot()); + assertEquals(wrapperList.get(i).getIngestDocument(), totalResults.get(i).getIngestDocument()); + assertNull(totalResults.get(i).getException()); + } + } + }); + } + private void assertStats(CompoundProcessor compoundProcessor, long count, long failed, long time) { assertStats(0, compoundProcessor, 0L, count, failed, time); } diff --git a/server/src/test/java/org/opensearch/ingest/IngestDocumentPreparer.java b/server/src/test/java/org/opensearch/ingest/IngestDocumentPreparer.java new file mode 100644 index 0000000000000..a02595df5589d --- /dev/null +++ b/server/src/test/java/org/opensearch/ingest/IngestDocumentPreparer.java @@ -0,0 +1,32 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.ingest; + +import java.util.HashMap; +import java.util.Map; + +public class IngestDocumentPreparer { + public static final String SHOULD_FAIL_KEY = "shouldFail"; + + public static IngestDocument createIngestDocument(boolean shouldFail) { + Map source = new HashMap<>(); + if (shouldFail) { + source.put(SHOULD_FAIL_KEY, true); + } + return new IngestDocument(source, new HashMap<>()); + } + + public static IngestDocumentWrapper createIngestDocumentWrapper(int slot) { + return createIngestDocumentWrapper(slot, false); + } + + public static IngestDocumentWrapper createIngestDocumentWrapper(int slot, boolean shouldFail) { + return new IngestDocumentWrapper(slot, createIngestDocument(shouldFail), null); + } +} diff --git a/server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTest.java b/server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTest.java new file mode 100644 index 0000000000000..7336b3ddcde96 --- /dev/null +++ b/server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTest.java @@ -0,0 +1,46 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.ingest; + +import org.junit.Before; +import org.opensearch.index.VersionType; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.HashMap; +import java.util.Map; + +public class IngestDocumentWrapperTest extends OpenSearchTestCase { + + private IngestDocument ingestDocument; + + private static final String INDEX = "index"; + private static final String ID = "id"; + private static final String ROUTING = "routing"; + private static final Long VERSION = 1L; + private static final VersionType VERSION_TYPE = VersionType.INTERNAL; + private static final String DOCUMENT_KEY = "foo"; + private static final String DOCUMENT_VALUE = "bar"; + private static final int SLOT = 12; + + @Before + public void setup() throws Exception { + super.setUp(); + Map document = new HashMap<>(); + document.put(DOCUMENT_KEY, DOCUMENT_VALUE); + ingestDocument = new IngestDocument(INDEX, ID, ROUTING, VERSION, VERSION_TYPE, document); + } + + public void testIngestDocumentWrapper() { + Exception ex = new RuntimeException("runtime exception"); + IngestDocumentWrapper wrapper = new IngestDocumentWrapper(SLOT, ingestDocument, ex); + assertEquals(wrapper.getSlot(), SLOT); + assertEquals(wrapper.getException(), ex); + assertEquals(wrapper.getIngestDocument(), ingestDocument); + } +} diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 23195eda525df..308c4f20fae9a 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -109,6 +109,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyString; @@ -132,6 +133,7 @@ public Map getProcessors(Processor.Parameters paramet }; private ThreadPool threadPool; + private BulkRequest mockBulkRequest; @Before public void setup() { @@ -139,6 +141,7 @@ public void setup() { ExecutorService executorService = OpenSearchExecutors.newDirectExecutorService(); when(threadPool.generic()).thenReturn(executorService); when(threadPool.executor(anyString())).thenReturn(executorService); + mockBulkRequest = mock(BulkRequest.class); } public void testIngestPlugin() { @@ -762,7 +765,8 @@ public String getType() { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + bulkRequest ); assertTrue(failure.get()); @@ -808,7 +812,8 @@ public void testExecuteBulkPipelineDoesNotExist() { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + bulkRequest ); verify(failureHandler, times(1)).accept( argThat((Integer item) -> item == 2), @@ -844,7 +849,8 @@ public void testExecuteSuccess() { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); verify(failureHandler, never()).accept(any(), any()); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); @@ -875,7 +881,8 @@ public void testExecuteEmptyPipeline() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); verify(failureHandler, never()).accept(any(), any()); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); @@ -934,7 +941,8 @@ public void testExecutePropagateAllMetadataUpdates() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); verify(processor).execute(any(), any()); verify(failureHandler, never()).accept(any(), any()); @@ -978,7 +986,8 @@ public void testExecuteFailure() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); verify(processor).execute(eqIndexTypeId(indexRequest.version(), indexRequest.versionType(), emptyMap()), any()); verify(failureHandler, times(1)).accept(eq(0), any(RuntimeException.class)); @@ -1036,7 +1045,8 @@ public void testExecuteSuccessWithOnFailure() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); verify(failureHandler, never()).accept(eq(0), any(IngestProcessorException.class)); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); @@ -1085,7 +1095,8 @@ public void testExecuteFailureWithNestedOnFailure() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); verify(processor).execute(eqIndexTypeId(indexRequest.version(), indexRequest.versionType(), emptyMap()), any()); verify(failureHandler, times(1)).accept(eq(0), any(RuntimeException.class)); @@ -1147,7 +1158,8 @@ public void testBulkRequestExecutionWithFailures() throws Exception { requestItemErrorHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + bulkRequest ); verify(requestItemErrorHandler, times(numIndexRequests)).accept(anyInt(), argThat(o -> o.getCause().equals(error))); @@ -1205,7 +1217,8 @@ public void testBulkRequestExecution() throws Exception { requestItemErrorHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + bulkRequest ); verify(requestItemErrorHandler, never()).accept(any(), any()); @@ -1273,7 +1286,8 @@ public void testStats() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); final IngestStats afterFirstRequestStats = ingestService.stats(); assertThat(afterFirstRequestStats.getPipelineStats().size(), equalTo(2)); @@ -1297,7 +1311,8 @@ public void testStats() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); final IngestStats afterSecondRequestStats = ingestService.stats(); assertThat(afterSecondRequestStats.getPipelineStats().size(), equalTo(2)); @@ -1326,7 +1341,8 @@ public void testStats() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); final IngestStats afterThirdRequestStats = ingestService.stats(); assertThat(afterThirdRequestStats.getPipelineStats().size(), equalTo(2)); @@ -1359,7 +1375,8 @@ public void testStats() throws Exception { failureHandler, completionHandler, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); final IngestStats afterForthRequestStats = ingestService.stats(); assertThat(afterForthRequestStats.getPipelineStats().size(), equalTo(2)); @@ -1457,7 +1474,8 @@ public String getDescription() { failureHandler, completionHandler, dropHandler, - Names.WRITE + Names.WRITE, + bulkRequest ); verify(failureHandler, never()).accept(any(), any()); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); @@ -1544,7 +1562,8 @@ public void testCBORParsing() throws Exception { (integer, e) -> {}, (thread, e) -> {}, indexReq -> {}, - Names.WRITE + Names.WRITE, + mockBulkRequest ); } @@ -1673,6 +1692,97 @@ public void testResolveRequestOrDefaultPipelineAndFinalPipeline() { } } + public void testExecuteBulkRequestInBatch() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + PutPipelineRequest putRequest = new PutPipelineRequest( + "_id", + new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), + MediaTypeRegistry.JSON + ); + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty + ClusterState previousClusterState = clusterState; + clusterState = IngestService.innerPut(putRequest, clusterState); + ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState)); + BulkRequest bulkRequest = new BulkRequest(); + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1") + .source(emptyMap()) + .setPipeline("_id") + .setFinalPipeline("_none"); + bulkRequest.add(indexRequest1); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2") + .source(emptyMap()) + .setPipeline("_id") + .setFinalPipeline("_none"); + bulkRequest.add(indexRequest2); + IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3") + .source(emptyMap()) + .setPipeline("_id") + .setFinalPipeline("_none"); + bulkRequest.add(indexRequest3); + IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4") + .source(emptyMap()) + .setPipeline("_id") + .setFinalPipeline("_none"); + bulkRequest.add(indexRequest4); + bulkRequest.batchIngestionOption("enabled"); + bulkRequest.maximumBatchSize(2); + @SuppressWarnings("unchecked") + final BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + final BiConsumer completionHandler = mock(BiConsumer.class); + ingestService.executeBulkRequest( + 4, + bulkRequest.requests(), + failureHandler, + completionHandler, + indexReq -> {}, + Names.WRITE, + bulkRequest + ); + verify(failureHandler, never()).accept(any(), any()); + verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + verify(mockCompoundProcessor, times(2)).batchExecute(any(), any()); + } + + public void testPrepareBatches_same_index_pipeline() { + IngestService.IndexRequestWrapper wrapper1 = createIndexRequestWrapper("index1", + Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper2 = createIndexRequestWrapper("index1", + Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper3 = createIndexRequestWrapper("index1", + Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper4 = createIndexRequestWrapper("index1", + Collections.singletonList("p1")); + List> batches = IngestService.prepareBatches(2, + Arrays.asList(wrapper1, wrapper2, wrapper3, wrapper4)); + assertEquals(2, batches.size()); + for (int i = 0; i < 2; ++i) { + assertEquals(2, batches.get(i).size()); + } + } + + public void testPrepareBatches_different_index_pipeline() { + IngestService.IndexRequestWrapper wrapper1 = createIndexRequestWrapper("index1", + Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper2 = createIndexRequestWrapper("index2", + Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper3 = createIndexRequestWrapper("index1", + Arrays.asList("p1", "p2")); + IngestService.IndexRequestWrapper wrapper4 = createIndexRequestWrapper("index1", + Collections.singletonList("p2")); + List> batches = IngestService.prepareBatches(2, + Arrays.asList(wrapper1, wrapper2, wrapper3, wrapper4)); + assertEquals(4, batches.size()); + } + + private IngestService.IndexRequestWrapper createIndexRequestWrapper(String index, List pipelines) { + IndexRequest indexRequest = new IndexRequest(index); + return new IngestService.IndexRequestWrapper(0, indexRequest, pipelines, true); + } + private IngestDocument eqIndexTypeId(final Map source) { return argThat(new IngestDocumentMatcher("_index", "_type", "_id", -3L, VersionType.INTERNAL, source)); } @@ -1719,6 +1829,13 @@ private CompoundProcessor mockCompoundProcessor() { handler.accept((IngestDocument) args.getArguments()[0], null); return null; }).when(processor).execute(any(), any()); + + doAnswer(args -> { + @SuppressWarnings("unchecked") + Consumer> handler = (Consumer) args.getArguments()[1]; + handler.accept((List) args.getArguments()[0]); + return null; + }).when(processor).batchExecute(any(), any()); return processor; } diff --git a/server/src/test/java/org/opensearch/ingest/ProcessorTests.java b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java new file mode 100644 index 0000000000000..936c9e250c6d9 --- /dev/null +++ b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java @@ -0,0 +1,70 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.ingest; + +import org.junit.Before; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Arrays; +import java.util.List; + +import static org.opensearch.ingest.IngestDocumentPreparer.SHOULD_FAIL_KEY; + +public class ProcessorTests extends OpenSearchTestCase { + private Processor processor; + private static final String FIELD_KEY = "result"; + private static final String FIELD_VALUE_PROCESSED = "processed"; + + @Before + public void setup() { + } + + public void test_batchExecute_success() { + processor = new FakeProcessor("type", "tag", "description", doc -> { + doc.setFieldValue(FIELD_KEY, FIELD_VALUE_PROCESSED); + }); + List wrapperList = Arrays.asList( + IngestDocumentPreparer.createIngestDocumentWrapper(1), + IngestDocumentPreparer.createIngestDocumentWrapper(2), + IngestDocumentPreparer.createIngestDocumentWrapper(3)); + processor.batchExecute(wrapperList, results -> { + assertEquals(3, results.size()); + for (IngestDocumentWrapper wrapper : results) { + assertNull(wrapper.getException()); + assertEquals(FIELD_VALUE_PROCESSED, wrapper.getIngestDocument().getFieldValue(FIELD_KEY, String.class)); + } + }); + } + + + public void test_batchExecute_exception() { + processor = new FakeProcessor("type", "tag", "description", doc -> { + if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { + throw new RuntimeException("fail"); + } + doc.setFieldValue(FIELD_KEY, FIELD_VALUE_PROCESSED); + }); + List wrapperList = Arrays.asList( + IngestDocumentPreparer.createIngestDocumentWrapper(1), + IngestDocumentPreparer.createIngestDocumentWrapper(2, true), + IngestDocumentPreparer.createIngestDocumentWrapper(3)); + processor.batchExecute(wrapperList, results -> { + assertEquals(3, results.size()); + for (IngestDocumentWrapper wrapper : results) { + if (wrapper.getSlot() == 2) { + assertNotNull(wrapper.getException()); + assertNull(wrapper.getIngestDocument()); + } else { + assertNull(wrapper.getException()); + assertEquals(FIELD_VALUE_PROCESSED, wrapper.getIngestDocument().getFieldValue(FIELD_KEY, String.class)); + } + } + }); + } +} From d5a4740fbede93e3b4796ffa4aa504c80f7c1434 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Fri, 19 Apr 2024 19:19:27 +0800 Subject: [PATCH 07/17] Add UT & yamlRest test, fix BulkRequest se/deserialization Signed-off-by: Liyun Xiu --- CHANGELOG.md | 1 + .../rest-api-spec/test/ingest/70_bulk.yml | 40 +++++++++++ .../resources/rest-api-spec/api/bulk.json | 8 +++ .../action/bulk/BatchIngestionOption.java | 20 +++++- .../opensearch/action/bulk/BulkRequest.java | 23 +++++-- .../opensearch/ingest/CompoundProcessor.java | 54 ++++++++------- .../org/opensearch/ingest/IngestService.java | 50 +++++++++----- .../java/org/opensearch/ingest/Pipeline.java | 1 - .../java/org/opensearch/ingest/Processor.java | 13 ++-- .../bulk/TransportBulkActionIngestTests.java | 2 +- .../ingest/CompoundProcessorTests.java | 14 ++-- ...t.java => IngestDocumentWrapperTests.java} | 4 +- .../opensearch/ingest/IngestServiceTests.java | 67 ++++++++----------- .../org/opensearch/ingest/ProcessorTests.java | 16 ++--- 14 files changed, 201 insertions(+), 112 deletions(-) rename server/src/test/java/org/opensearch/ingest/{IngestDocumentWrapperTest.java => IngestDocumentWrapperTests.java} (95%) diff --git a/CHANGELOG.md b/CHANGELOG.md index cc493a4f4618c..6b310689d9bee 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -25,6 +25,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Streaming Indexing] Ensure support of the new transport by security plugin ([#13174](https://github.com/opensearch-project/OpenSearch/pull/13174)) - Add cluster setting to dynamically configure the buckets for filter rewrite optimization. ([#13179](https://github.com/opensearch-project/OpenSearch/pull/13179)) - [Remote Store] Add capability of doing refresh as determined by the translog ([#12992](https://github.com/opensearch-project/OpenSearch/pull/12992)) +- Add support for batch ingestion capability ([#12457](https://github.com/opensearch-project/OpenSearch/issues/12457)) ### Dependencies - Bump `org.apache.commons:commons-configuration2` from 2.10.0 to 2.10.1 ([#12896](https://github.com/opensearch-project/OpenSearch/pull/12896)) diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml index d7be48a92908c..c8bae345a120b 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml @@ -167,3 +167,43 @@ teardown: index: test_index id: test_id3 - match: { _source: {"f1": "v2", "f2": 47, "field1": "value1"}} + +--- +"Test bulk API with batch enabled": + - skip: + version: " - 2.13.99" + reason: "Added in 2.14.0" + + - do: + bulk: + refresh: true + batch_ingestion_option: "enabled" + maximum_batch_size: 2 + pipeline: "pipeline1" + body: + - '{"index": {"_index": "test_index", "_id": "test_id1"}}' + - '{"text": "text1"}' + - '{"index": {"_index": "test_index", "_id": "test_id2"}}' + - '{"text": "text2"}' + - '{"index": {"_index": "test_index", "_id": "test_id3"}}' + - '{"text": "text3"}' + - '{"index": {"_index": "test_index", "_id": "test_id4"}}' + - '{"text": "text4"}' + - '{"index": {"_index": "test_index", "_id": "test_id5", "pipeline": "pipeline2"}}' + - '{"text": "text5"}' + - '{"index": {"_index": "test_index", "_id": "test_id6", "pipeline": "pipeline2"}}' + - '{"text": "text6"}' + + - match: { errors: false } + + - do: + get: + index: test_index + id: test_id5 + - match: { _source: {"text": "text5", "field2": "value2"}} + + - do: + get: + index: test_index + id: test_id3 + - match: { _source: { "text": "text3", "field1": "value1" } } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json index bb066cd131480..3b9140285f673 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json @@ -74,6 +74,14 @@ "require_alias": { "type": "boolean", "description": "Sets require_alias for all incoming documents. Defaults to unset (false)" + }, + "maximum_batch_size": { + "type": "int", + "description": "set the batch size" + }, + "batch_ingestion_option": { + "type": "string", + "description": "A signal to enable batch ingestion" } }, "body":{ diff --git a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java index e3e4621f868a5..43a5e39848bb9 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java +++ b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java @@ -9,12 +9,17 @@ package org.opensearch.action.bulk; import org.opensearch.common.annotation.PublicApi; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; + +import java.io.IOException; /** * An enum for batch ingestion option. */ -@PublicApi(since = "2.15.0") -public enum BatchIngestionOption { +@PublicApi(since = "2.14.0") +public enum BatchIngestionOption implements Writeable { NONE("disabled"), ENABLED("enabled"); @@ -34,9 +39,18 @@ static BatchIngestionOption from(String value) { return option; } } - if ("".equals(value)) { + if (value == null || value.isEmpty()) { return NONE; } throw new IllegalArgumentException("Unknown value for batch ingestion option: [" + value + "]."); } + + public static BatchIngestionOption readFrom(StreamInput in) throws IOException { + return BatchIngestionOption.values()[in.readByte()]; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeByte((byte) ordinal()); + } } diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index a1931f5ee85af..0468122c39976 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -34,6 +34,7 @@ import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; +import org.opensearch.Version; import org.opensearch.action.ActionRequest; import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.CompositeIndicesRequest; @@ -80,7 +81,7 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(BulkRequest.class); private static final int REQUEST_OVERHEAD = 50; - + private static final Version MINIMAL_VERSION_SUPPORT_BATCH = Version.V_2_14_0; /** * Requests that are part of this request. It is only possible to add things that are both {@link ActionRequest}s and * {@link WriteRequest}s to this but java doesn't support syntax to declare that everything in the array has both types so we declare @@ -96,8 +97,8 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private String globalRouting; private String globalIndex; private Boolean globalRequireAlias; - private BatchIngestionOption batchIngestionOption; - private Integer maximumBatchSize; + private BatchIngestionOption batchIngestionOption = BatchIngestionOption.NONE; + private Integer maximumBatchSize = 1; private long sizeInBytes = 0; @@ -109,6 +110,10 @@ public BulkRequest(StreamInput in) throws IOException { requests.addAll(in.readList(i -> DocWriteRequest.readDocumentRequest(null, i))); refreshPolicy = RefreshPolicy.readFrom(in); timeout = in.readTimeValue(); + if (in.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { + batchIngestionOption = BatchIngestionOption.readFrom(in); + maximumBatchSize = in.readInt(); + } } public BulkRequest(@Nullable String globalIndex) { @@ -353,7 +358,7 @@ public final BulkRequest timeout(TimeValue timeout) { * @param batchOption a string input from request * @return {@link BulkRequest} */ - public final BulkRequest batchIngestionOption(String batchOption) { + public BulkRequest batchIngestionOption(String batchOption) { this.batchIngestionOption = BatchIngestionOption.from(batchOption); return this; } @@ -362,7 +367,7 @@ public final BulkRequest batchIngestionOption(String batchOption) { * Get batch ingestion option * @return {@link BatchIngestionOption} */ - public final BatchIngestionOption batchIngestionOption() { + public BatchIngestionOption batchIngestionOption() { return this.batchIngestionOption; } @@ -371,7 +376,7 @@ public final BatchIngestionOption batchIngestionOption() { * @param size maximum batch size from input * @return {@link BulkRequest} */ - public final BulkRequest maximumBatchSize(int size) { + public BulkRequest maximumBatchSize(int size) { if (size > 1) { this.maximumBatchSize = size; } else { @@ -384,7 +389,7 @@ public final BulkRequest maximumBatchSize(int size) { * Get maximum batch size * @return maximum batch size */ - public final int maximumBatchSize() { + public int maximumBatchSize() { return this.maximumBatchSize; } @@ -495,6 +500,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(requests, DocWriteRequest::writeDocumentRequest); refreshPolicy.writeTo(out); out.writeTimeValue(timeout); + if (out.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { + batchIngestionOption.writeTo(out); + out.writeInt(maximumBatchSize); + } } @Override diff --git a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java index e0a5eb3597382..c1e202ab023e1 100644 --- a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java @@ -154,13 +154,15 @@ public void execute(IngestDocument ingestDocument, BiConsumer ingestDocumentWrappers, - Consumer> handler) { + public void batchExecute(List ingestDocumentWrappers, Consumer> handler) { innerBatchExecute(0, ingestDocumentWrappers, handler); } - public void innerBatchExecute(int currentProcessor, List ingestDocumentWrappers, - Consumer> handler) { + public void innerBatchExecute( + int currentProcessor, + List ingestDocumentWrappers, + Consumer> handler + ) { if (currentProcessor == processorsWithMetrics.size()) { handler.accept(ingestDocumentWrappers); return; @@ -174,8 +176,7 @@ public void innerBatchExecute(int currentProcessor, List // Use synchronization to ensure batches are processed by processors in sequential order AtomicInteger counter = new AtomicInteger(size); List allResults = Collections.synchronizedList(new ArrayList<>()); - Map slotToWrapperMap = - createSlotIngestDocumentWrapperMap(ingestDocumentWrappers); + Map slotToWrapperMap = createSlotIngestDocumentWrapperMap(ingestDocumentWrappers); processor.batchExecute(ingestDocumentWrappers, results -> { if (results.isEmpty()) return; allResults.addAll(results); @@ -194,15 +195,18 @@ public void innerBatchExecute(int currentProcessor, List if (ignoreFailure) { documentsToContinue.add(originalDocumentWrapper); } else { - IngestProcessorException compoundProcessorException = - newCompoundProcessorException(resultDocumentWrapper.getException(), - processor, - originalDocumentWrapper.getIngestDocument() - ); - documentsWithException.add(new IngestDocumentWrapper(resultDocumentWrapper.getSlot(), - originalDocumentWrapper.getIngestDocument(), - compoundProcessorException - )); + IngestProcessorException compoundProcessorException = newCompoundProcessorException( + resultDocumentWrapper.getException(), + processor, + originalDocumentWrapper.getIngestDocument() + ); + documentsWithException.add( + new IngestDocumentWrapper( + resultDocumentWrapper.getSlot(), + originalDocumentWrapper.getIngestDocument(), + compoundProcessorException + ) + ); } } else { if (resultDocumentWrapper.getIngestDocument() == null) { @@ -227,18 +231,23 @@ public void innerBatchExecute(int currentProcessor, List if (onFailureProcessors.isEmpty()) { handler.accept(documentsWithException); } else { - documentsWithException.forEach(doc -> executeOnFailureAsync(0, doc.getIngestDocument(), - (IngestProcessorException) doc.getException(), (result, ex) -> { - handler.accept(Collections.singletonList(new IngestDocumentWrapper(doc.getSlot(), result, ex))); - })); + documentsWithException.forEach( + doc -> executeOnFailureAsync( + 0, + doc.getIngestDocument(), + (IngestProcessorException) doc.getException(), + (result, ex) -> { + handler.accept(Collections.singletonList(new IngestDocumentWrapper(doc.getSlot(), result, ex))); + } + ) + ); } } } }); } - void innerExecute(int currentProcessor, IngestDocument ingestDocument, - BiConsumer handler) { + void innerExecute(int currentProcessor, IngestDocument ingestDocument, BiConsumer handler) { if (currentProcessor == processorsWithMetrics.size()) { handler.accept(ingestDocument, null); return; @@ -354,8 +363,7 @@ static IngestProcessorException newCompoundProcessorException(Exception e, Proce return exception; } - private Map createSlotIngestDocumentWrapperMap( - List ingestDocumentWrappers) { + private Map createSlotIngestDocumentWrapperMap(List ingestDocumentWrappers) { Map slotIngestDocumentWrapperMap = new HashMap<>(); for (IngestDocumentWrapper ingestDocumentWrapper : ingestDocumentWrappers) { slotIngestDocumentWrapperMap.put(ingestDocumentWrapper.getSlot(), ingestDocumentWrapper); diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 5f80c2cd20971..df0651356a539 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -529,8 +529,7 @@ protected void doRun() { BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); int batchSize = originalBulkRequest.maximumBatchSize(); if (shouldExecuteBulkRequestInBatch(batchOption, originalBulkRequest.requests().size(), batchSize)) { - runBulkRequestInBatch(numberOfActionRequests, actionRequests, onFailure, onCompletion, onDropped, - originalBulkRequest); + runBulkRequestInBatch(numberOfActionRequests, actionRequests, onFailure, onCompletion, onDropped, originalBulkRequest); return; } @@ -587,12 +586,14 @@ protected void doRun() { }); } - private void runBulkRequestInBatch(int numberOfActionRequests, + private void runBulkRequestInBatch( + int numberOfActionRequests, Iterable> actionRequests, BiConsumer onFailure, BiConsumer onCompletion, IntConsumer onDropped, - BulkRequest originalBulkRequest) { + BulkRequest originalBulkRequest + ) { final Thread originalThread = Thread.currentThread(); final AtomicInteger counter = new AtomicInteger(numberOfActionRequests); @@ -641,9 +642,8 @@ private void runBulkRequestInBatch(int numberOfActionRequests, logger.debug("batchSize: {}, batches: {}", batchSize, batches.size()); for (List batch : batches) { - executePipelinesInBatchRequests(batch.stream() - .map(IndexRequestWrapper::getSlot) - .collect(Collectors.toList()), + executePipelinesInBatchRequests( + batch.stream().map(IndexRequestWrapper::getSlot).collect(Collectors.toList()), batch.get(0).getPipelines().iterator(), batch.get(0).isHasFinalPipeline(), batch.stream().map(IndexRequestWrapper::getIndexRequest).collect(Collectors.toList()), @@ -688,9 +688,11 @@ static List> prepareBatches(int batchSize, List> batchedIndexRequests = new ArrayList<>(); for (Map.Entry> indexRequestsPerKey : indexRequestsPerIndexAndPipelines.entrySet()) { for (int i = 0; i < indexRequestsPerKey.getValue().size(); i += batchSize) { - batchedIndexRequests.add(new ArrayList<>( - indexRequestsPerKey.getValue().subList(i, - i + Math.min(batchSize, indexRequestsPerKey.getValue().size() - i)))); + batchedIndexRequests.add( + new ArrayList<>( + indexRequestsPerKey.getValue().subList(i, i + Math.min(batchSize, indexRequestsPerKey.getValue().size() - i)) + ) + ); } } return batchedIndexRequests; @@ -739,7 +741,17 @@ private void executePipelinesInBatchRequests( final Thread originalThread ) { if (indexRequests.size() == 1) { - executePipelines(slots.get(0), it, hasFinalPipeline, indexRequests.get(0), onDropped, onFailure, counter, onCompletion, originalThread); + executePipelines( + slots.get(0), + it, + hasFinalPipeline, + indexRequests.get(0), + onDropped, + onFailure, + counter, + onCompletion, + originalThread + ); return; } while (it.hasNext()) { @@ -1088,7 +1100,8 @@ private void innerBatchExecute( for (IngestDocumentWrapper ingestDocumentWrapper : succeeded) { updateIndexRequestWithIngestDocument( slotToindexRequestMap.get(ingestDocumentWrapper.getSlot()), - ingestDocumentWrapper.getIngestDocument()); + ingestDocumentWrapper.getIngestDocument() + ); } } handler.accept(results); @@ -1303,16 +1316,21 @@ public static void updateIndexRequestWithIngestDocument(IndexRequest indexReques } private static IngestDocument toIngestDocument(IndexRequest indexRequest) { - return new IngestDocument(indexRequest.index(), indexRequest.id(), indexRequest.routing(), - indexRequest.version(), indexRequest.versionType(), indexRequest.sourceAsMap()); + return new IngestDocument( + indexRequest.index(), + indexRequest.id(), + indexRequest.routing(), + indexRequest.version(), + indexRequest.versionType(), + indexRequest.sourceAsMap() + ); } private static IngestDocumentWrapper toIngestDocumentWrapper(int slot, IndexRequest indexRequest) { return new IngestDocumentWrapper(slot, toIngestDocument(indexRequest), null); } - private static Map createSlotIndexRequestMap(List slots, - List indexRequests) { + private static Map createSlotIndexRequestMap(List slots, List indexRequests) { Map slotIndexRequestMap = new HashMap<>(); for (int i = 0; i < slots.size(); ++i) { slotIndexRequestMap.put(slots.get(i), indexRequests.get(i)); diff --git a/server/src/main/java/org/opensearch/ingest/Pipeline.java b/server/src/main/java/org/opensearch/ingest/Pipeline.java index 105da5f72632f..117c680710900 100644 --- a/server/src/main/java/org/opensearch/ingest/Pipeline.java +++ b/server/src/main/java/org/opensearch/ingest/Pipeline.java @@ -34,7 +34,6 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.common.Nullable; -import org.opensearch.common.collect.Tuple; import org.opensearch.common.metrics.OperationMetrics; import org.opensearch.script.ScriptService; diff --git a/server/src/main/java/org/opensearch/ingest/Processor.java b/server/src/main/java/org/opensearch/ingest/Processor.java index 1a2d8f734619a..02c17fe50bfaa 100644 --- a/server/src/main/java/org/opensearch/ingest/Processor.java +++ b/server/src/main/java/org/opensearch/ingest/Processor.java @@ -85,8 +85,7 @@ default void execute(IngestDocument ingestDocument, BiConsumer ingestDocumentWrappers, - Consumer> handler) { + default void batchExecute(List ingestDocumentWrappers, Consumer> handler) { if (ingestDocumentWrappers.isEmpty()) { handler.accept(Collections.emptyList()); return; @@ -99,9 +98,13 @@ default void batchExecute(List ingestDocumentWrappers, } } - private void innerExecute(int slot, IngestDocumentWrapper ingestDocumentWrapper, - AtomicArray results, AtomicInteger counter, - Consumer> handler) { + private void innerExecute( + int slot, + IngestDocumentWrapper ingestDocumentWrapper, + AtomicArray results, + AtomicInteger counter, + Consumer> handler + ) { execute(ingestDocumentWrapper.getIngestDocument(), (doc, ex) -> { results.set(slot, new IngestDocumentWrapper(ingestDocumentWrapper.getSlot(), doc, ex)); if (counter.decrementAndGet() == 0) { diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java index 65f96f12a571b..da9156ccdb71a 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionIngestTests.java @@ -342,7 +342,7 @@ public void testIngestLocal() throws Exception { completionHandler.capture(), any(), eq(Names.WRITE), - bulkRequest + eq(bulkRequest) ); completionHandler.getValue().accept(null, exception); assertTrue(failureCalled.get()); diff --git a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java index 5d39be2665a4d..b3dd710d30ab9 100644 --- a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java @@ -53,6 +53,7 @@ import java.util.function.LongSupplier; import static java.util.Collections.singletonList; +import static org.opensearch.ingest.IngestDocumentPreparer.SHOULD_FAIL_KEY; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; @@ -63,7 +64,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import static org.opensearch.ingest.IngestDocumentPreparer.SHOULD_FAIL_KEY; public class CompoundProcessorTests extends OpenSearchTestCase { private IngestDocument ingestDocument; @@ -440,7 +440,8 @@ public void testBatchExecute_happyCase() { List wrapperList = Arrays.asList( IngestDocumentPreparer.createIngestDocumentWrapper(1), IngestDocumentPreparer.createIngestDocumentWrapper(2), - IngestDocumentPreparer.createIngestDocumentWrapper(3)); + IngestDocumentPreparer.createIngestDocumentWrapper(3) + ); TestProcessor firstProcessor = new TestProcessor(doc -> {}); TestProcessor secondProcessor = new TestProcessor(doc -> {}); LongSupplier relativeTimeProvider = mock(LongSupplier.class); @@ -470,7 +471,8 @@ public void testBatchExecute_ignoreFailure() { List wrapperList = Arrays.asList( IngestDocumentPreparer.createIngestDocumentWrapper(1), IngestDocumentPreparer.createIngestDocumentWrapper(2, true), - IngestDocumentPreparer.createIngestDocumentWrapper(3, true)); + IngestDocumentPreparer.createIngestDocumentWrapper(3, true) + ); TestProcessor firstProcessor = new TestProcessor(doc -> { if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { throw new RuntimeException("fail"); @@ -507,7 +509,8 @@ public void testBatchExecute_exception_no_onFailureProcessor() { List wrapperList = Arrays.asList( IngestDocumentPreparer.createIngestDocumentWrapper(1), IngestDocumentPreparer.createIngestDocumentWrapper(2, true), - IngestDocumentPreparer.createIngestDocumentWrapper(3, true)); + IngestDocumentPreparer.createIngestDocumentWrapper(3, true) + ); TestProcessor firstProcessor = new TestProcessor(doc -> { if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { throw new RuntimeException("fail"); @@ -552,7 +555,8 @@ public void testBatchExecute_exception_with_onFailureProcessor() { List wrapperList = Arrays.asList( IngestDocumentPreparer.createIngestDocumentWrapper(1), IngestDocumentPreparer.createIngestDocumentWrapper(2, true), - IngestDocumentPreparer.createIngestDocumentWrapper(3, true)); + IngestDocumentPreparer.createIngestDocumentWrapper(3, true) + ); TestProcessor firstProcessor = new TestProcessor(doc -> { if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { throw new RuntimeException("fail"); diff --git a/server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTest.java b/server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTests.java similarity index 95% rename from server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTest.java rename to server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTests.java index 7336b3ddcde96..9d09cd80abd05 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTest.java +++ b/server/src/test/java/org/opensearch/ingest/IngestDocumentWrapperTests.java @@ -8,14 +8,14 @@ package org.opensearch.ingest; -import org.junit.Before; import org.opensearch.index.VersionType; import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; import java.util.HashMap; import java.util.Map; -public class IngestDocumentWrapperTest extends OpenSearchTestCase { +public class IngestDocumentWrapperTests extends OpenSearchTestCase { private IngestDocument ingestDocument; diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 308c4f20fae9a..86be3e1d88948 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -38,6 +38,7 @@ import org.opensearch.ResourceNotFoundException; import org.opensearch.Version; import org.opensearch.action.DocWriteRequest; +import org.opensearch.action.bulk.BatchIngestionOption; import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.TransportBulkAction; import org.opensearch.action.delete.DeleteRequest; @@ -109,7 +110,6 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; -import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyString; @@ -117,6 +117,7 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; @@ -142,6 +143,8 @@ public void setup() { when(threadPool.generic()).thenReturn(executorService); when(threadPool.executor(anyString())).thenReturn(executorService); mockBulkRequest = mock(BulkRequest.class); + lenient().when(mockBulkRequest.batchIngestionOption()).thenReturn(BatchIngestionOption.NONE); + lenient().when(mockBulkRequest.maximumBatchSize()).thenReturn(1); } public void testIngestPlugin() { @@ -1707,25 +1710,13 @@ public void testExecuteBulkRequestInBatch() { clusterState = IngestService.innerPut(putRequest, clusterState); ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState)); BulkRequest bulkRequest = new BulkRequest(); - IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1") - .source(emptyMap()) - .setPipeline("_id") - .setFinalPipeline("_none"); + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest1); - IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2") - .source(emptyMap()) - .setPipeline("_id") - .setFinalPipeline("_none"); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest2); - IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3") - .source(emptyMap()) - .setPipeline("_id") - .setFinalPipeline("_none"); + IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest3); - IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4") - .source(emptyMap()) - .setPipeline("_id") - .setFinalPipeline("_none"); + IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest4); bulkRequest.batchIngestionOption("enabled"); bulkRequest.maximumBatchSize(2); @@ -1748,34 +1739,30 @@ public void testExecuteBulkRequestInBatch() { } public void testPrepareBatches_same_index_pipeline() { - IngestService.IndexRequestWrapper wrapper1 = createIndexRequestWrapper("index1", - Collections.singletonList("p1")); - IngestService.IndexRequestWrapper wrapper2 = createIndexRequestWrapper("index1", - Collections.singletonList("p1")); - IngestService.IndexRequestWrapper wrapper3 = createIndexRequestWrapper("index1", - Collections.singletonList("p1")); - IngestService.IndexRequestWrapper wrapper4 = createIndexRequestWrapper("index1", - Collections.singletonList("p1")); - List> batches = IngestService.prepareBatches(2, - Arrays.asList(wrapper1, wrapper2, wrapper3, wrapper4)); - assertEquals(2, batches.size()); + IngestService.IndexRequestWrapper wrapper1 = createIndexRequestWrapper("index1", Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper2 = createIndexRequestWrapper("index1", Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper3 = createIndexRequestWrapper("index1", Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper4 = createIndexRequestWrapper("index1", Collections.singletonList("p1")); + List> batches = IngestService.prepareBatches( + 2, + Arrays.asList(wrapper1, wrapper2, wrapper3, wrapper4) + ); + assertEquals(2, batches.size()); for (int i = 0; i < 2; ++i) { - assertEquals(2, batches.get(i).size()); + assertEquals(2, batches.get(i).size()); } } public void testPrepareBatches_different_index_pipeline() { - IngestService.IndexRequestWrapper wrapper1 = createIndexRequestWrapper("index1", - Collections.singletonList("p1")); - IngestService.IndexRequestWrapper wrapper2 = createIndexRequestWrapper("index2", - Collections.singletonList("p1")); - IngestService.IndexRequestWrapper wrapper3 = createIndexRequestWrapper("index1", - Arrays.asList("p1", "p2")); - IngestService.IndexRequestWrapper wrapper4 = createIndexRequestWrapper("index1", - Collections.singletonList("p2")); - List> batches = IngestService.prepareBatches(2, - Arrays.asList(wrapper1, wrapper2, wrapper3, wrapper4)); - assertEquals(4, batches.size()); + IngestService.IndexRequestWrapper wrapper1 = createIndexRequestWrapper("index1", Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper2 = createIndexRequestWrapper("index2", Collections.singletonList("p1")); + IngestService.IndexRequestWrapper wrapper3 = createIndexRequestWrapper("index1", Arrays.asList("p1", "p2")); + IngestService.IndexRequestWrapper wrapper4 = createIndexRequestWrapper("index1", Collections.singletonList("p2")); + List> batches = IngestService.prepareBatches( + 2, + Arrays.asList(wrapper1, wrapper2, wrapper3, wrapper4) + ); + assertEquals(4, batches.size()); } private IngestService.IndexRequestWrapper createIndexRequestWrapper(String index, List pipelines) { diff --git a/server/src/test/java/org/opensearch/ingest/ProcessorTests.java b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java index 936c9e250c6d9..4fd53661f801a 100644 --- a/server/src/test/java/org/opensearch/ingest/ProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java @@ -8,8 +8,8 @@ package org.opensearch.ingest; -import org.junit.Before; import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; import java.util.Arrays; import java.util.List; @@ -22,17 +22,15 @@ public class ProcessorTests extends OpenSearchTestCase { private static final String FIELD_VALUE_PROCESSED = "processed"; @Before - public void setup() { - } + public void setup() {} public void test_batchExecute_success() { - processor = new FakeProcessor("type", "tag", "description", doc -> { - doc.setFieldValue(FIELD_KEY, FIELD_VALUE_PROCESSED); - }); + processor = new FakeProcessor("type", "tag", "description", doc -> { doc.setFieldValue(FIELD_KEY, FIELD_VALUE_PROCESSED); }); List wrapperList = Arrays.asList( IngestDocumentPreparer.createIngestDocumentWrapper(1), IngestDocumentPreparer.createIngestDocumentWrapper(2), - IngestDocumentPreparer.createIngestDocumentWrapper(3)); + IngestDocumentPreparer.createIngestDocumentWrapper(3) + ); processor.batchExecute(wrapperList, results -> { assertEquals(3, results.size()); for (IngestDocumentWrapper wrapper : results) { @@ -42,7 +40,6 @@ public void test_batchExecute_success() { }); } - public void test_batchExecute_exception() { processor = new FakeProcessor("type", "tag", "description", doc -> { if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { @@ -53,7 +50,8 @@ public void test_batchExecute_exception() { List wrapperList = Arrays.asList( IngestDocumentPreparer.createIngestDocumentWrapper(1), IngestDocumentPreparer.createIngestDocumentWrapper(2, true), - IngestDocumentPreparer.createIngestDocumentWrapper(3)); + IngestDocumentPreparer.createIngestDocumentWrapper(3) + ); processor.batchExecute(wrapperList, results -> { assertEquals(3, results.size()); for (IngestDocumentWrapper wrapper : results) { From e6c3b9f8b12796fb829cf10eace3b3e68455c7b2 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Sun, 21 Apr 2024 19:18:16 +0800 Subject: [PATCH 08/17] Add missing java docs Signed-off-by: Liyun Xiu --- .../action/bulk/BatchIngestionOption.java | 6 ++--- .../opensearch/action/bulk/BulkRequest.java | 2 +- .../opensearch/ingest/CompoundProcessor.java | 13 ++++++++--- .../org/opensearch/ingest/IngestService.java | 22 +++++++++---------- .../java/org/opensearch/ingest/Pipeline.java | 10 +++++++++ .../java/org/opensearch/ingest/Processor.java | 8 +++++++ 6 files changed, 43 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java index 43a5e39848bb9..6f79bef30752c 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java +++ b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java @@ -34,14 +34,14 @@ public String getValue() { } static BatchIngestionOption from(String value) { + if (value == null || value.isBlank()) { + return NONE; + } for (BatchIngestionOption option : values()) { if (option.getValue().equals(value)) { return option; } } - if (value == null || value.isEmpty()) { - return NONE; - } throw new IllegalArgumentException("Unknown value for batch ingestion option: [" + value + "]."); } diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 0468122c39976..b3c0a149d596d 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -98,7 +98,7 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private String globalIndex; private Boolean globalRequireAlias; private BatchIngestionOption batchIngestionOption = BatchIngestionOption.NONE; - private Integer maximumBatchSize = 1; + private int maximumBatchSize = 1; private long sizeInBytes = 0; diff --git a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java index c1e202ab023e1..89f2be829b22b 100644 --- a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java @@ -158,7 +158,14 @@ public void batchExecute(List ingestDocumentWrappers, Con innerBatchExecute(0, ingestDocumentWrappers, handler); } - public void innerBatchExecute( + /** + * Internal logic to process documents with current processor. + * + * @param currentProcessor index of processor to process batched documents + * @param ingestDocumentWrappers {@link List } batched documents to be processed + * @param handler callback function + */ + void innerBatchExecute( int currentProcessor, List ingestDocumentWrappers, Consumer> handler @@ -180,6 +187,7 @@ public void innerBatchExecute( processor.batchExecute(ingestDocumentWrappers, results -> { if (results.isEmpty()) return; allResults.addAll(results); + // counter equals to 0 means all documents are processed and called back. if (counter.addAndGet(-results.size()) == 0) { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); metric.afterN(allResults.size(), ingestTimeInMillis); @@ -188,6 +196,7 @@ public void innerBatchExecute( List documentsWithException = new ArrayList<>(); List documentsToContinue = new ArrayList<>(); int totalFailed = 0; + // iterate all results to categorize them to: to continue, to drop, with exception for (IngestDocumentWrapper resultDocumentWrapper : allResults) { IngestDocumentWrapper originalDocumentWrapper = slotToWrapperMap.get(resultDocumentWrapper.getSlot()); if (resultDocumentWrapper.getException() != null) { @@ -222,11 +231,9 @@ public void innerBatchExecute( if (!documentsDropped.isEmpty()) { handler.accept(documentsDropped); } - if (!documentsToContinue.isEmpty()) { innerBatchExecute(currentProcessor + 1, documentsToContinue, handler); } - if (!documentsWithException.isEmpty()) { if (onFailureProcessors.isEmpty()) { handler.accept(documentsWithException); diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index df0651356a539..4cb987a194a76 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -731,7 +731,7 @@ public boolean isHasFinalPipeline() { private void executePipelinesInBatchRequests( final List slots, - final Iterator it, + final Iterator pipelineIterator, final boolean hasFinalPipeline, final List indexRequests, final IntConsumer onDropped, @@ -743,7 +743,7 @@ private void executePipelinesInBatchRequests( if (indexRequests.size() == 1) { executePipelines( slots.get(0), - it, + pipelineIterator, hasFinalPipeline, indexRequests.get(0), onDropped, @@ -754,8 +754,8 @@ private void executePipelinesInBatchRequests( ); return; } - while (it.hasNext()) { - final String pipelineId = it.next(); + while (pipelineIterator.hasNext()) { + final String pipelineId = pipelineIterator.next(); try { PipelineHolder holder = pipelines.get(pipelineId); if (holder == null) { @@ -781,14 +781,14 @@ private void executePipelinesInBatchRequests( } } - Iterator newIt = it; + Iterator newPipelineIterator = pipelineIterator; boolean newHasFinalPipeline = hasFinalPipeline; // indexRequests are grouped for the same index and same pipelines String newIndex = indexRequests.get(0).indices()[0]; // handle index change case if (Objects.equals(originalIndex, newIndex) == false) { - if (hasFinalPipeline && it.hasNext() == false) { + if (hasFinalPipeline && pipelineIterator.hasNext() == false) { totalMetrics.failed(); for (int slot : slots) { onFailure.accept( @@ -798,24 +798,24 @@ private void executePipelinesInBatchRequests( } } else { // Drain old it so it's not looped over - it.forEachRemaining($ -> {}); + pipelineIterator.forEachRemaining($ -> {}); for (IndexRequest indexRequest : indexRequests) { indexRequest.isPipelineResolved(false); resolvePipelines(null, indexRequest, state.metadata()); if (IngestService.NOOP_PIPELINE_NAME.equals(indexRequest.getFinalPipeline()) == false) { - newIt = Collections.singleton(indexRequest.getFinalPipeline()).iterator(); + newPipelineIterator = Collections.singleton(indexRequest.getFinalPipeline()).iterator(); newHasFinalPipeline = true; } else { - newIt = Collections.emptyIterator(); + newPipelineIterator = Collections.emptyIterator(); } } } } - if (newIt.hasNext()) { + if (newPipelineIterator.hasNext()) { executePipelinesInBatchRequests( slots, - newIt, + newPipelineIterator, newHasFinalPipeline, indexRequests, onDropped, diff --git a/server/src/main/java/org/opensearch/ingest/Pipeline.java b/server/src/main/java/org/opensearch/ingest/Pipeline.java index 117c680710900..b8f068edd3e59 100644 --- a/server/src/main/java/org/opensearch/ingest/Pipeline.java +++ b/server/src/main/java/org/opensearch/ingest/Pipeline.java @@ -203,6 +203,16 @@ public OperationMetrics getMetrics() { return metrics; } + /** + * Modifies the data of batched multiple documents to be indexed based on the processor this pipeline holds + *

+ * If {@code null} is returned then this document will be dropped and not indexed, otherwise + * this document will be kept and indexed. Document and the exception happened during processing are kept in + * IngestDocumentWrapper and callback to upper level. + * + * @param ingestDocumentWrappers {@link List} a list of wrapped IngestDocument to ingest. + * @param handler callback with IngestDocument result and exception wrapped in IngestDocumentWrapper. + */ public void batchExecute(List ingestDocumentWrappers, Consumer> handler) { final long startTimeInNanos = relativeTimeProvider.getAsLong(); int size = ingestDocumentWrappers.size(); diff --git a/server/src/main/java/org/opensearch/ingest/Processor.java b/server/src/main/java/org/opensearch/ingest/Processor.java index 02c17fe50bfaa..5829e35f1b8b1 100644 --- a/server/src/main/java/org/opensearch/ingest/Processor.java +++ b/server/src/main/java/org/opensearch/ingest/Processor.java @@ -85,6 +85,14 @@ default void execute(IngestDocument ingestDocument, BiConsumer} a list of wrapped IngestDocument + * @param handler callback with IngestDocument result and exception wrapped in IngestDocumentWrapper. + */ default void batchExecute(List ingestDocumentWrappers, Consumer> handler) { if (ingestDocumentWrappers.isEmpty()) { handler.accept(Collections.emptyList()); From 8ef01cd3097c3d48d4ae7522f3132db1ff813d10 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Wed, 24 Apr 2024 14:52:27 +0800 Subject: [PATCH 09/17] Remove Writable from BatchIngestionOption Signed-off-by: Liyun Xiu --- .../action/bulk/BatchIngestionOption.java | 16 +--------------- .../org/opensearch/action/bulk/BulkRequest.java | 4 ++-- .../org/opensearch/ingest/CompoundProcessor.java | 3 ++- .../org/opensearch/ingest/IngestService.java | 1 + .../java/org/opensearch/ingest/Pipeline.java | 2 +- .../java/org/opensearch/ingest/Processor.java | 2 +- 6 files changed, 8 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java index 6f79bef30752c..3f4c49c674b06 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java +++ b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java @@ -9,17 +9,12 @@ package org.opensearch.action.bulk; import org.opensearch.common.annotation.PublicApi; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.core.common.io.stream.Writeable; - -import java.io.IOException; /** * An enum for batch ingestion option. */ @PublicApi(since = "2.14.0") -public enum BatchIngestionOption implements Writeable { +public enum BatchIngestionOption { NONE("disabled"), ENABLED("enabled"); @@ -44,13 +39,4 @@ static BatchIngestionOption from(String value) { } throw new IllegalArgumentException("Unknown value for batch ingestion option: [" + value + "]."); } - - public static BatchIngestionOption readFrom(StreamInput in) throws IOException { - return BatchIngestionOption.values()[in.readByte()]; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeByte((byte) ordinal()); - } } diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index b3c0a149d596d..1062d17b6b0a4 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -111,7 +111,7 @@ public BulkRequest(StreamInput in) throws IOException { refreshPolicy = RefreshPolicy.readFrom(in); timeout = in.readTimeValue(); if (in.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { - batchIngestionOption = BatchIngestionOption.readFrom(in); + batchIngestionOption = in.readEnum(BatchIngestionOption.class); maximumBatchSize = in.readInt(); } } @@ -501,7 +501,7 @@ public void writeTo(StreamOutput out) throws IOException { refreshPolicy.writeTo(out); out.writeTimeValue(timeout); if (out.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { - batchIngestionOption.writeTo(out); + out.writeEnum(batchIngestionOption); out.writeInt(maximumBatchSize); } } diff --git a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java index 89f2be829b22b..64d71691bf818 100644 --- a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java @@ -162,7 +162,7 @@ public void batchExecute(List ingestDocumentWrappers, Con * Internal logic to process documents with current processor. * * @param currentProcessor index of processor to process batched documents - * @param ingestDocumentWrappers {@link List } batched documents to be processed + * @param ingestDocumentWrappers batched documents to be processed * @param handler callback function */ void innerBatchExecute( @@ -251,6 +251,7 @@ void innerBatchExecute( } } } + assert counter.get() >= 0; }); } diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 4cb987a194a76..158ab5af9e51c 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -1106,6 +1106,7 @@ private void innerBatchExecute( } handler.accept(results); } + assert counter.get() >= 0; }); } diff --git a/server/src/main/java/org/opensearch/ingest/Pipeline.java b/server/src/main/java/org/opensearch/ingest/Pipeline.java index b8f068edd3e59..708416cfca3b7 100644 --- a/server/src/main/java/org/opensearch/ingest/Pipeline.java +++ b/server/src/main/java/org/opensearch/ingest/Pipeline.java @@ -210,7 +210,7 @@ public OperationMetrics getMetrics() { * this document will be kept and indexed. Document and the exception happened during processing are kept in * IngestDocumentWrapper and callback to upper level. * - * @param ingestDocumentWrappers {@link List} a list of wrapped IngestDocument to ingest. + * @param ingestDocumentWrappers a list of wrapped IngestDocument to ingest. * @param handler callback with IngestDocument result and exception wrapped in IngestDocumentWrapper. */ public void batchExecute(List ingestDocumentWrappers, Consumer> handler) { diff --git a/server/src/main/java/org/opensearch/ingest/Processor.java b/server/src/main/java/org/opensearch/ingest/Processor.java index 5829e35f1b8b1..9af1104502047 100644 --- a/server/src/main/java/org/opensearch/ingest/Processor.java +++ b/server/src/main/java/org/opensearch/ingest/Processor.java @@ -90,7 +90,7 @@ default void execute(IngestDocument ingestDocument, BiConsumer} a list of wrapped IngestDocument + * @param ingestDocumentWrappers a list of wrapped IngestDocument * @param handler callback with IngestDocument result and exception wrapped in IngestDocumentWrapper. */ default void batchExecute(List ingestDocumentWrappers, Consumer> handler) { From 696fe740bb14df7e70b57cf07f218b41df606c6c Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Wed, 24 Apr 2024 21:02:49 +0800 Subject: [PATCH 10/17] Add more UTs Signed-off-by: Liyun Xiu --- .../org/opensearch/ingest/IngestService.java | 6 +- .../ingest/CompoundProcessorTests.java | 46 ++++ .../opensearch/ingest/IngestServiceTests.java | 242 +++++++++++++++++- .../org/opensearch/ingest/ProcessorTests.java | 8 + 4 files changed, 289 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 158ab5af9e51c..42fad2e21d508 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -1083,7 +1083,7 @@ private void innerBatchExecute( List succeeded = new ArrayList<>(); List dropped = new ArrayList<>(); List exceptions = new ArrayList<>(); - for (IngestDocumentWrapper result : results) { + for (IngestDocumentWrapper result : allResults) { if (result.getException() != null) { exceptions.add(result); } else if (result.getIngestDocument() == null) { @@ -1104,7 +1104,7 @@ private void innerBatchExecute( ); } } - handler.accept(results); + handler.accept(allResults); } assert counter.get() >= 0; }); @@ -1316,7 +1316,7 @@ public static void updateIndexRequestWithIngestDocument(IndexRequest indexReques indexRequest.source(ingestDocument.getSourceAndMetadata(), indexRequest.getContentType()); } - private static IngestDocument toIngestDocument(IndexRequest indexRequest) { + static IngestDocument toIngestDocument(IndexRequest indexRequest) { return new IngestDocument( indexRequest.index(), indexRequest.id(), diff --git a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java index b3dd710d30ab9..aad6063bd3f4d 100644 --- a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java @@ -467,6 +467,52 @@ public void testBatchExecute_happyCase() { }); } + public void testBatchExecute_documentToDrop() { + List wrapperList = Arrays.asList( + IngestDocumentPreparer.createIngestDocumentWrapper(1), + IngestDocumentPreparer.createIngestDocumentWrapper(2, true), + IngestDocumentPreparer.createIngestDocumentWrapper(3) + ); + TestProcessor firstProcessor = new TestProcessor("", "", "", doc -> { + if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { + return null; + } + return doc; + }); + TestProcessor secondProcessor = new TestProcessor(doc -> {}); + LongSupplier relativeTimeProvider = mock(LongSupplier.class); + CompoundProcessor compoundProcessor = new CompoundProcessor( + false, + Arrays.asList(firstProcessor, secondProcessor), + null, + relativeTimeProvider + ); + + AtomicInteger callCounter = new AtomicInteger(); + List totalResults = Collections.synchronizedList(new ArrayList<>()); + compoundProcessor.batchExecute(wrapperList, results -> { + totalResults.addAll(results); + if (callCounter.addAndGet(results.size()) == 3) { + assertEquals(firstProcessor.getInvokedCounter(), wrapperList.size()); + assertEquals(secondProcessor.getInvokedCounter(), wrapperList.size() - 1); + assertEquals(totalResults.size(), wrapperList.size()); + OperationStats stats = compoundProcessor.getProcessorsWithMetrics().get(0).v2().createStats(); + assertEquals(0, stats.getCurrent()); + assertEquals(3, stats.getCount()); + totalResults.sort(Comparator.comparingInt(IngestDocumentWrapper::getSlot)); + for (int i = 0; i < wrapperList.size(); ++i) { + assertEquals(wrapperList.get(i).getSlot(), totalResults.get(i).getSlot()); + if (2 == wrapperList.get(i).getSlot()) { + assertNull(totalResults.get(i).getIngestDocument()); + } else { + assertEquals(wrapperList.get(i).getIngestDocument(), totalResults.get(i).getIngestDocument()); + } + assertEquals(wrapperList.get(i).getException(), totalResults.get(i).getException()); + } + } + }); + } + public void testBatchExecute_ignoreFailure() { List wrapperList = Arrays.asList( IngestDocumentPreparer.createIngestDocumentWrapper(1), diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 86be3e1d88948..c59bbc5451958 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -1700,21 +1700,13 @@ public void testExecuteBulkRequestInBatch() { IngestService ingestService = createWithProcessors( Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) ); - PutPipelineRequest putRequest = new PutPipelineRequest( - "_id", - new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), - MediaTypeRegistry.JSON - ); - ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty - ClusterState previousClusterState = clusterState; - clusterState = IngestService.innerPut(putRequest, clusterState); - ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState)); + createPipeline("_id", ingestService); BulkRequest bulkRequest = new BulkRequest(); IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest1); IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest2); - IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3").source(emptyMap()).setPipeline("_none").setFinalPipeline("_id"); bulkRequest.add(indexRequest3); IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest4); @@ -1736,6 +1728,216 @@ public void testExecuteBulkRequestInBatch() { verify(failureHandler, never()).accept(any(), any()); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); verify(mockCompoundProcessor, times(2)).batchExecute(any(), any()); + verify(mockCompoundProcessor, never()).execute(any(), any()); + } + + public void testExecuteBulkRequestInBatchWithDefaultAndFinalPipeline() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + ClusterState clusterState = createPipeline("_id", ingestService); + createPipeline("_final", ingestService, clusterState); + BulkRequest bulkRequest = new BulkRequest(); + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_final"); + bulkRequest.add(indexRequest1); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_final"); + bulkRequest.add(indexRequest2); + IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3").source(emptyMap()).setPipeline("_id").setFinalPipeline("_final"); + bulkRequest.add(indexRequest3); + IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_final"); + bulkRequest.add(indexRequest4); + bulkRequest.batchIngestionOption("enabled"); + bulkRequest.maximumBatchSize(2); + @SuppressWarnings("unchecked") + final BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + final BiConsumer completionHandler = mock(BiConsumer.class); + ingestService.executeBulkRequest( + 4, + bulkRequest.requests(), + failureHandler, + completionHandler, + indexReq -> {}, + Names.WRITE, + bulkRequest + ); + verify(failureHandler, never()).accept(any(), any()); + verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + verify(mockCompoundProcessor, times(4)).batchExecute(any(), any()); + verify(mockCompoundProcessor, never()).execute(any(), any()); + } + + public void testExecuteBulkRequestInBatchFallbackWithOneDocument() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + createPipeline("_id", ingestService); + BulkRequest bulkRequest = new BulkRequest(); + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest1); + bulkRequest.batchIngestionOption("enabled"); + bulkRequest.maximumBatchSize(2); + @SuppressWarnings("unchecked") + final BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + final BiConsumer completionHandler = mock(BiConsumer.class); + ingestService.executeBulkRequest( + 1, + bulkRequest.requests(), + failureHandler, + completionHandler, + indexReq -> {}, + Names.WRITE, + bulkRequest + ); + verify(failureHandler, never()).accept(any(), any()); + verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + verify(mockCompoundProcessor, never()).batchExecute(any(), any()); + verify(mockCompoundProcessor, times(1)).execute(any(), any()); + } + + public void testExecuteBulkRequestInBatchNoValidPipeline() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + createPipeline("_id", ingestService); + BulkRequest bulkRequest = new BulkRequest(); + // will not be handled as not valid document type + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_none").setFinalPipeline("_none"); + bulkRequest.add(indexRequest1); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_none").setFinalPipeline("_none"); + bulkRequest.add(indexRequest2); + bulkRequest.batchIngestionOption("enabled"); + bulkRequest.maximumBatchSize(2); + @SuppressWarnings("unchecked") + final BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + final BiConsumer completionHandler = mock(BiConsumer.class); + ingestService.executeBulkRequest( + 2, + bulkRequest.requests(), + failureHandler, + completionHandler, + indexReq -> {}, + Names.WRITE, + bulkRequest + ); + verify(failureHandler, never()).accept(any(), any()); + verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + verify(mockCompoundProcessor, never()).batchExecute(any(), any()); + verify(mockCompoundProcessor, never()).execute(any(), any()); + } + + public void testExecuteBulkRequestInBatchNoValidDocument() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + createPipeline("_id", ingestService); + BulkRequest bulkRequest = new BulkRequest(); + // will not be handled as not valid document type + bulkRequest.add(new DeleteRequest("_index", "_id")); + bulkRequest.add(new DeleteRequest("_index", "_id")); + bulkRequest.batchIngestionOption("enabled"); + bulkRequest.maximumBatchSize(2); + @SuppressWarnings("unchecked") + final BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + final BiConsumer completionHandler = mock(BiConsumer.class); + ingestService.executeBulkRequest( + 2, + bulkRequest.requests(), + failureHandler, + completionHandler, + indexReq -> {}, + Names.WRITE, + bulkRequest + ); + verify(failureHandler, never()).accept(any(), any()); + verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + verify(mockCompoundProcessor, never()).batchExecute(any(), any()); + verify(mockCompoundProcessor, never()).execute(any(), any()); + } + + public void testExecuteBulkRequestInBatchWithException() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + doThrow(new RuntimeException()).when(mockCompoundProcessor).batchExecute(any(), any()); + createPipeline("_id", ingestService); + BulkRequest bulkRequest = new BulkRequest(); + // will not be handled as not valid document type + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest1); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest2); + bulkRequest.batchIngestionOption("enabled"); + bulkRequest.maximumBatchSize(2); + @SuppressWarnings("unchecked") + final BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + final BiConsumer completionHandler = mock(BiConsumer.class); + ingestService.executeBulkRequest( + 2, + bulkRequest.requests(), + failureHandler, + completionHandler, + indexReq -> {}, + Names.WRITE, + bulkRequest + ); + verify(failureHandler, times(2)).accept(any(), any()); + verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + verify(mockCompoundProcessor, times(1)).batchExecute(any(), any()); + verify(mockCompoundProcessor, never()).execute(any(), any()); + } + + public void testExecuteBulkRequestInBatchWithExceptionInCallback() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + createPipeline("_id", ingestService); + BulkRequest bulkRequest = new BulkRequest(); + // will not be handled as not valid document type + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest1); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest2); + bulkRequest.batchIngestionOption("enabled"); + bulkRequest.maximumBatchSize(2); + + List results = Arrays.asList( + new IngestDocumentWrapper(0, IngestService.toIngestDocument(indexRequest1), null), + new IngestDocumentWrapper(1, null, new RuntimeException())); + doAnswer(args -> { + @SuppressWarnings("unchecked") + Consumer> handler = (Consumer) args.getArguments()[1]; + handler.accept(results); + return null; + }).when(mockCompoundProcessor).batchExecute(any(), any()); + + @SuppressWarnings("unchecked") + final BiConsumer failureHandler = mock(BiConsumer.class); + @SuppressWarnings("unchecked") + final BiConsumer completionHandler = mock(BiConsumer.class); + ingestService.executeBulkRequest( + 2, + bulkRequest.requests(), + failureHandler, + completionHandler, + indexReq -> {}, + Names.WRITE, + bulkRequest + ); + verify(failureHandler, times(1)).accept(any(), any()); + verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + verify(mockCompoundProcessor, times(1)).batchExecute(any(), any()); + verify(mockCompoundProcessor, never()).execute(any(), any()); } public void testPrepareBatches_same_index_pipeline() { @@ -1862,4 +2064,24 @@ private void assertStats(OperationStats stats, long count, long failed, long tim private OperationStats getPipelineStats(List pipelineStats, String id) { return pipelineStats.stream().filter(p1 -> p1.getPipelineId().equals(id)).findFirst().map(p2 -> p2.getStats()).orElse(null); } + + private ClusterState createPipeline(String pipeline, IngestService ingestService) { + return createPipeline(pipeline, ingestService, null); + } + + private ClusterState createPipeline(String pipeline, IngestService ingestService, ClusterState previousState) { + PutPipelineRequest putRequest = new PutPipelineRequest( + pipeline, + new BytesArray("{\"processors\": [{\"mock\" : {}}]}"), + MediaTypeRegistry.JSON + ); + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); // Start empty + if (previousState != null) { + clusterState = previousState; + } + ClusterState previousClusterState = clusterState; + clusterState = IngestService.innerPut(putRequest, clusterState); + ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState)); + return clusterState; + } } diff --git a/server/src/test/java/org/opensearch/ingest/ProcessorTests.java b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java index 4fd53661f801a..8574ee8ad5c93 100644 --- a/server/src/test/java/org/opensearch/ingest/ProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java @@ -12,6 +12,7 @@ import org.junit.Before; import java.util.Arrays; +import java.util.Collections; import java.util.List; import static org.opensearch.ingest.IngestDocumentPreparer.SHOULD_FAIL_KEY; @@ -40,6 +41,13 @@ public void test_batchExecute_success() { }); } + public void test_batchExecute_empty() { + processor = new FakeProcessor("type", "tag", "description", doc -> { doc.setFieldValue(FIELD_KEY, FIELD_VALUE_PROCESSED); }); + processor.batchExecute(Collections.emptyList(), results -> { + assertEquals(0, results.size()); + }); + } + public void test_batchExecute_exception() { processor = new FakeProcessor("type", "tag", "description", doc -> { if (doc.hasField(SHOULD_FAIL_KEY) && doc.getFieldValue(SHOULD_FAIL_KEY, Boolean.class)) { From af65dbeff5946d9588e7c6fc0e96a8a0e2e3b90d Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Wed, 24 Apr 2024 21:24:34 +0800 Subject: [PATCH 11/17] Fix spotlesscheck Signed-off-by: Liyun Xiu --- .../org/opensearch/ingest/IngestServiceTests.java | 13 ++++++++++--- .../java/org/opensearch/ingest/ProcessorTests.java | 4 +--- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index c59bbc5451958..5ea6067b40c17 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -1806,9 +1806,15 @@ public void testExecuteBulkRequestInBatchNoValidPipeline() { createPipeline("_id", ingestService); BulkRequest bulkRequest = new BulkRequest(); // will not be handled as not valid document type - IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_none").setFinalPipeline("_none"); + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1") + .source(emptyMap()) + .setPipeline("_none") + .setFinalPipeline("_none"); bulkRequest.add(indexRequest1); - IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_none").setFinalPipeline("_none"); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2") + .source(emptyMap()) + .setPipeline("_none") + .setFinalPipeline("_none"); bulkRequest.add(indexRequest2); bulkRequest.batchIngestionOption("enabled"); bulkRequest.maximumBatchSize(2); @@ -1913,7 +1919,8 @@ public void testExecuteBulkRequestInBatchWithExceptionInCallback() { List results = Arrays.asList( new IngestDocumentWrapper(0, IngestService.toIngestDocument(indexRequest1), null), - new IngestDocumentWrapper(1, null, new RuntimeException())); + new IngestDocumentWrapper(1, null, new RuntimeException()) + ); doAnswer(args -> { @SuppressWarnings("unchecked") Consumer> handler = (Consumer) args.getArguments()[1]; diff --git a/server/src/test/java/org/opensearch/ingest/ProcessorTests.java b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java index 8574ee8ad5c93..d6ef3be73adb8 100644 --- a/server/src/test/java/org/opensearch/ingest/ProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/ProcessorTests.java @@ -43,9 +43,7 @@ public void test_batchExecute_success() { public void test_batchExecute_empty() { processor = new FakeProcessor("type", "tag", "description", doc -> { doc.setFieldValue(FIELD_KEY, FIELD_VALUE_PROCESSED); }); - processor.batchExecute(Collections.emptyList(), results -> { - assertEquals(0, results.size()); - }); + processor.batchExecute(Collections.emptyList(), results -> { assertEquals(0, results.size()); }); } public void test_batchExecute_exception() { From 81afff201515ff48797079b705946de9fa8da2ec Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Mon, 29 Apr 2024 23:18:47 +0800 Subject: [PATCH 12/17] Rename parameter name to batch_size Signed-off-by: Liyun Xiu --- CHANGELOG.md | 2 +- .../rest-api-spec/test/ingest/70_bulk.yml | 2 +- .../resources/rest-api-spec/api/bulk.json | 4 ++-- .../opensearch/action/bulk/BulkRequest.java | 24 +++++++++---------- .../org/opensearch/ingest/IngestService.java | 4 ++-- .../rest/action/document/RestBulkAction.java | 2 +- .../opensearch/ingest/IngestServiceTests.java | 16 ++++++------- 7 files changed, 27 insertions(+), 27 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 18725d1e52417..9bd6ad2ab1c19 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -26,7 +26,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add cluster setting to dynamically configure the buckets for filter rewrite optimization. ([#13179](https://github.com/opensearch-project/OpenSearch/pull/13179)) - [Tiered Caching] Add a dynamic setting to disable/enable disk cache. ([#13373](https://github.com/opensearch-project/OpenSearch/pull/13373)) - [Remote Store] Add capability of doing refresh as determined by the translog ([#12992](https://github.com/opensearch-project/OpenSearch/pull/12992)) -- Add support for batch ingestion capability ([#12457](https://github.com/opensearch-project/OpenSearch/issues/12457)) +- [Batch Ingestion] Add `batch_ingestion_option` and `batch_size` parameters in `_bulk` API. ([#12457](https://github.com/opensearch-project/OpenSearch/issues/12457)) - [Tiered caching] Make Indices Request Cache Stale Key Mgmt Threshold setting dynamic ([#12941](https://github.com/opensearch-project/OpenSearch/pull/12941)) - Batch mode for async fetching shard information in GatewayAllocator for unassigned shards ([#8746](https://github.com/opensearch-project/OpenSearch/pull/8746)) diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml index c8bae345a120b..8b72b6cb77feb 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml @@ -178,7 +178,7 @@ teardown: bulk: refresh: true batch_ingestion_option: "enabled" - maximum_batch_size: 2 + batch_size: 2 pipeline: "pipeline1" body: - '{"index": {"_index": "test_index", "_id": "test_id1"}}' diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json index 3b9140285f673..d9902cf0f6d3d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json @@ -75,9 +75,9 @@ "type": "boolean", "description": "Sets require_alias for all incoming documents. Defaults to unset (false)" }, - "maximum_batch_size": { + "batch_size": { "type": "int", - "description": "set the batch size" + "description": "Sets the batch size" }, "batch_ingestion_option": { "type": "string", diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 1062d17b6b0a4..8cbca890b3584 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -98,7 +98,7 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private String globalIndex; private Boolean globalRequireAlias; private BatchIngestionOption batchIngestionOption = BatchIngestionOption.NONE; - private int maximumBatchSize = 1; + private int batchSize = 1; private long sizeInBytes = 0; @@ -112,7 +112,7 @@ public BulkRequest(StreamInput in) throws IOException { timeout = in.readTimeValue(); if (in.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { batchIngestionOption = in.readEnum(BatchIngestionOption.class); - maximumBatchSize = in.readInt(); + batchSize = in.readInt(); } } @@ -372,25 +372,25 @@ public BatchIngestionOption batchIngestionOption() { } /** - * Set maximum batch size - * @param size maximum batch size from input + * Set batch size + * @param size batch size from input * @return {@link BulkRequest} */ - public BulkRequest maximumBatchSize(int size) { + public BulkRequest batchSize(int size) { if (size > 1) { - this.maximumBatchSize = size; + this.batchSize = size; } else { - this.maximumBatchSize = 1; + this.batchSize = 1; } return this; } /** - * Get maximum batch size - * @return maximum batch size + * Get batch size + * @return batch size */ - public int maximumBatchSize() { - return this.maximumBatchSize; + public int batchSize() { + return this.batchSize; } /** @@ -502,7 +502,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeTimeValue(timeout); if (out.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { out.writeEnum(batchIngestionOption); - out.writeInt(maximumBatchSize); + out.writeInt(batchSize); } } diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 42fad2e21d508..3326c458b5752 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -527,7 +527,7 @@ public void onFailure(Exception e) { @Override protected void doRun() { BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); - int batchSize = originalBulkRequest.maximumBatchSize(); + int batchSize = originalBulkRequest.batchSize(); if (shouldExecuteBulkRequestInBatch(batchOption, originalBulkRequest.requests().size(), batchSize)) { runBulkRequestInBatch(numberOfActionRequests, actionRequests, onFailure, onCompletion, onDropped, originalBulkRequest); return; @@ -637,7 +637,7 @@ private void runBulkRequestInBatch( i++; } - int batchSize = originalBulkRequest.maximumBatchSize(); + int batchSize = originalBulkRequest.batchSize(); List> batches = prepareBatches(batchSize, indexRequestWrappers); logger.debug("batchSize: {}, batches: {}", batchSize, batches.size()); diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java index 709a76bd9921a..d090a22db3418 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java @@ -98,7 +98,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); bulkRequest.setRefreshPolicy(request.param("refresh")); bulkRequest.batchIngestionOption(request.param("batch_ingestion_option", "")); - bulkRequest.maximumBatchSize(request.paramAsInt("maximum_batch_size", 1)); + bulkRequest.batchSize(request.paramAsInt("batch_size", 1)); bulkRequest.add( request.requiredContent(), defaultIndex, diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 5ea6067b40c17..4608dc75742aa 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -144,7 +144,7 @@ public void setup() { when(threadPool.executor(anyString())).thenReturn(executorService); mockBulkRequest = mock(BulkRequest.class); lenient().when(mockBulkRequest.batchIngestionOption()).thenReturn(BatchIngestionOption.NONE); - lenient().when(mockBulkRequest.maximumBatchSize()).thenReturn(1); + lenient().when(mockBulkRequest.batchSize()).thenReturn(1); } public void testIngestPlugin() { @@ -1711,7 +1711,7 @@ public void testExecuteBulkRequestInBatch() { IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest4); bulkRequest.batchIngestionOption("enabled"); - bulkRequest.maximumBatchSize(2); + bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @SuppressWarnings("unchecked") @@ -1748,7 +1748,7 @@ public void testExecuteBulkRequestInBatchWithDefaultAndFinalPipeline() { IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_final"); bulkRequest.add(indexRequest4); bulkRequest.batchIngestionOption("enabled"); - bulkRequest.maximumBatchSize(2); + bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @SuppressWarnings("unchecked") @@ -1778,7 +1778,7 @@ public void testExecuteBulkRequestInBatchFallbackWithOneDocument() { IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest1); bulkRequest.batchIngestionOption("enabled"); - bulkRequest.maximumBatchSize(2); + bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @SuppressWarnings("unchecked") @@ -1817,7 +1817,7 @@ public void testExecuteBulkRequestInBatchNoValidPipeline() { .setFinalPipeline("_none"); bulkRequest.add(indexRequest2); bulkRequest.batchIngestionOption("enabled"); - bulkRequest.maximumBatchSize(2); + bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @SuppressWarnings("unchecked") @@ -1848,7 +1848,7 @@ public void testExecuteBulkRequestInBatchNoValidDocument() { bulkRequest.add(new DeleteRequest("_index", "_id")); bulkRequest.add(new DeleteRequest("_index", "_id")); bulkRequest.batchIngestionOption("enabled"); - bulkRequest.maximumBatchSize(2); + bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @SuppressWarnings("unchecked") @@ -1882,7 +1882,7 @@ public void testExecuteBulkRequestInBatchWithException() { IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest2); bulkRequest.batchIngestionOption("enabled"); - bulkRequest.maximumBatchSize(2); + bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @SuppressWarnings("unchecked") @@ -1915,7 +1915,7 @@ public void testExecuteBulkRequestInBatchWithExceptionInCallback() { IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest2); bulkRequest.batchIngestionOption("enabled"); - bulkRequest.maximumBatchSize(2); + bulkRequest.batchSize(2); List results = Arrays.asList( new IngestDocumentWrapper(0, IngestService.toIngestDocument(indexRequest1), null), From 574d2ff512b0434a23f1298074c05da80b76c9ad Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 30 Apr 2024 01:39:53 +0800 Subject: [PATCH 13/17] Add more rest yaml tests & update rest spec Signed-off-by: Liyun Xiu --- .../rest-api-spec/test/ingest/70_bulk.yml | 115 +++++++++++++++++- .../resources/rest-api-spec/api/bulk.json | 4 +- 2 files changed, 116 insertions(+), 3 deletions(-) diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml index 8b72b6cb77feb..d64f3b05f5117 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml @@ -169,7 +169,7 @@ teardown: - match: { _source: {"f1": "v2", "f2": 47, "field1": "value1"}} --- -"Test bulk API with batch enabled": +"Test bulk API with batch enabled happy case": - skip: version: " - 2.13.99" reason: "Added in 2.14.0" @@ -207,3 +207,116 @@ teardown: index: test_index id: test_id3 - match: { _source: { "text": "text3", "field1": "value1" } } + +--- +"Test bulk API with batch_size missing": + - skip: + version: " - 2.13.99" + reason: "Added in 2.14.0" + + - do: + bulk: + refresh: true + batch_ingestion_option: "enabled" + pipeline: "pipeline1" + body: + - '{"index": {"_index": "test_index", "_id": "test_id1"}}' + - '{"text": "text1"}' + - '{"index": {"_index": "test_index", "_id": "test_id2"}}' + - '{"text": "text2"}' + + - match: { errors: false } + + - do: + get: + index: test_index + id: test_id1 + - match: { _source: { "text": "text1", "field1": "value1" } } + + - do: + get: + index: test_index + id: test_id2 + - match: { _source: { "text": "text2", "field1": "value1" } } + +--- +"Test bulk API with batch_ingestion_option missing": + - skip: + version: " - 2.13.99" + reason: "Added in 2.14.0" + + - do: + bulk: + refresh: true + batch_size: 2 + pipeline: "pipeline1" + body: + - '{"index": {"_index": "test_index", "_id": "test_id1"}}' + - '{"text": "text1"}' + - '{"index": {"_index": "test_index", "_id": "test_id2"}}' + - '{"text": "text2"}' + + - match: { errors: false } + + - do: + get: + index: test_index + id: test_id1 + - match: { _source: { "text": "text1", "field1": "value1" } } + + - do: + get: + index: test_index + id: test_id2 + - match: { _source: { "text": "text2", "field1": "value1" } } + +--- +"Test bulk API with invalid batch_size": + - skip: + version: " - 2.13.99" + reason: "Added in 2.14.0" + + - do: + bulk: + refresh: true + batch_size: -1 + batch_ingestion_option: "enabled" + pipeline: "pipeline1" + body: + - '{"index": {"_index": "test_index", "_id": "test_id1"}}' + - '{"text": "text1"}' + - '{"index": {"_index": "test_index", "_id": "test_id2"}}' + - '{"text": "text2"}' + + - match: { errors: false } + + - do: + get: + index: test_index + id: test_id1 + - match: { _source: { "text": "text1", "field1": "value1" } } + + - do: + get: + index: test_index + id: test_id2 + - match: { _source: { "text": "text2", "field1": "value1" } } + +--- +"Test bulk API with invalid batch_ingestion_option": + - skip: + version: " - 2.13.99" + reason: "Added in 2.14.0" + + - do: + catch: bad_request + bulk: + refresh: true + batch_size: 2 + batch_ingestion_option: "random text" + pipeline: "pipeline1" + body: + - '{"index": {"_index": "test_index", "_id": "test_id1"}}' + - '{"text": "text1"}' + - '{"index": {"_index": "test_index", "_id": "test_id2"}}' + - '{"text": "text2"}' diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json index d9902cf0f6d3d..e0dac24db2c43 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json @@ -77,11 +77,11 @@ }, "batch_size": { "type": "int", - "description": "Sets the batch size" + "description": "Sets the batch size with batch_ingestion_option" }, "batch_ingestion_option": { "type": "string", - "description": "A signal to enable batch ingestion" + "description": "Type of batch ingestion to use" } }, "body":{ From e2fb58552bf17ea99cecbd9f803fdea214d02d9d Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 30 Apr 2024 02:18:51 +0800 Subject: [PATCH 14/17] Remove batch_ingestion_option and only use batch_size Signed-off-by: Liyun Xiu --- CHANGELOG.md | 2 +- .../rest-api-spec/test/ingest/70_bulk.yml | 53 ------------------- .../resources/rest-api-spec/api/bulk.json | 6 +-- .../action/bulk/BatchIngestionOption.java | 42 --------------- .../opensearch/action/bulk/BulkRequest.java | 21 -------- .../org/opensearch/ingest/IngestService.java | 8 ++- .../rest/action/document/RestBulkAction.java | 1 - .../opensearch/ingest/IngestServiceTests.java | 9 ---- 8 files changed, 5 insertions(+), 137 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 9bd6ad2ab1c19..d0bb4ffd9683e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -26,7 +26,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add cluster setting to dynamically configure the buckets for filter rewrite optimization. ([#13179](https://github.com/opensearch-project/OpenSearch/pull/13179)) - [Tiered Caching] Add a dynamic setting to disable/enable disk cache. ([#13373](https://github.com/opensearch-project/OpenSearch/pull/13373)) - [Remote Store] Add capability of doing refresh as determined by the translog ([#12992](https://github.com/opensearch-project/OpenSearch/pull/12992)) -- [Batch Ingestion] Add `batch_ingestion_option` and `batch_size` parameters in `_bulk` API. ([#12457](https://github.com/opensearch-project/OpenSearch/issues/12457)) +- [Batch Ingestion] Add `batch_size` to `_bulk` API. ([#12457](https://github.com/opensearch-project/OpenSearch/issues/12457)) - [Tiered caching] Make Indices Request Cache Stale Key Mgmt Threshold setting dynamic ([#12941](https://github.com/opensearch-project/OpenSearch/pull/12941)) - Batch mode for async fetching shard information in GatewayAllocator for unassigned shards ([#8746](https://github.com/opensearch-project/OpenSearch/pull/8746)) diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml index d64f3b05f5117..d319f68a75c95 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml @@ -177,7 +177,6 @@ teardown: - do: bulk: refresh: true - batch_ingestion_option: "enabled" batch_size: 2 pipeline: "pipeline1" body: @@ -217,38 +216,6 @@ teardown: - do: bulk: refresh: true - batch_ingestion_option: "enabled" - pipeline: "pipeline1" - body: - - '{"index": {"_index": "test_index", "_id": "test_id1"}}' - - '{"text": "text1"}' - - '{"index": {"_index": "test_index", "_id": "test_id2"}}' - - '{"text": "text2"}' - - - match: { errors: false } - - - do: - get: - index: test_index - id: test_id1 - - match: { _source: { "text": "text1", "field1": "value1" } } - - - do: - get: - index: test_index - id: test_id2 - - match: { _source: { "text": "text2", "field1": "value1" } } - ---- -"Test bulk API with batch_ingestion_option missing": - - skip: - version: " - 2.13.99" - reason: "Added in 2.14.0" - - - do: - bulk: - refresh: true - batch_size: 2 pipeline: "pipeline1" body: - '{"index": {"_index": "test_index", "_id": "test_id1"}}' @@ -280,7 +247,6 @@ teardown: bulk: refresh: true batch_size: -1 - batch_ingestion_option: "enabled" pipeline: "pipeline1" body: - '{"index": {"_index": "test_index", "_id": "test_id1"}}' @@ -301,22 +267,3 @@ teardown: index: test_index id: test_id2 - match: { _source: { "text": "text2", "field1": "value1" } } - ---- -"Test bulk API with invalid batch_ingestion_option": - - skip: - version: " - 2.13.99" - reason: "Added in 2.14.0" - - - do: - catch: bad_request - bulk: - refresh: true - batch_size: 2 - batch_ingestion_option: "random text" - pipeline: "pipeline1" - body: - - '{"index": {"_index": "test_index", "_id": "test_id1"}}' - - '{"text": "text1"}' - - '{"index": {"_index": "test_index", "_id": "test_id2"}}' - - '{"text": "text2"}' diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json index e0dac24db2c43..e0566b811ff07 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json @@ -77,11 +77,7 @@ }, "batch_size": { "type": "int", - "description": "Sets the batch size with batch_ingestion_option" - }, - "batch_ingestion_option": { - "type": "string", - "description": "Type of batch ingestion to use" + "description": "Sets the batch size" } }, "body":{ diff --git a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java b/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java deleted file mode 100644 index 3f4c49c674b06..0000000000000 --- a/server/src/main/java/org/opensearch/action/bulk/BatchIngestionOption.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.action.bulk; - -import org.opensearch.common.annotation.PublicApi; - -/** - * An enum for batch ingestion option. - */ -@PublicApi(since = "2.14.0") -public enum BatchIngestionOption { - NONE("disabled"), - ENABLED("enabled"); - - private final String value; - - public String getValue() { - return this.value; - } - - BatchIngestionOption(String value) { - this.value = value; - } - - static BatchIngestionOption from(String value) { - if (value == null || value.isBlank()) { - return NONE; - } - for (BatchIngestionOption option : values()) { - if (option.getValue().equals(value)) { - return option; - } - } - throw new IllegalArgumentException("Unknown value for batch ingestion option: [" + value + "]."); - } -} diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 8cbca890b3584..11a38f679fb70 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -97,7 +97,6 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private String globalRouting; private String globalIndex; private Boolean globalRequireAlias; - private BatchIngestionOption batchIngestionOption = BatchIngestionOption.NONE; private int batchSize = 1; private long sizeInBytes = 0; @@ -111,7 +110,6 @@ public BulkRequest(StreamInput in) throws IOException { refreshPolicy = RefreshPolicy.readFrom(in); timeout = in.readTimeValue(); if (in.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { - batchIngestionOption = in.readEnum(BatchIngestionOption.class); batchSize = in.readInt(); } } @@ -353,24 +351,6 @@ public final BulkRequest timeout(TimeValue timeout) { return this; } - /** - * Convert string version of batch option and convert it to {@link BatchIngestionOption} - * @param batchOption a string input from request - * @return {@link BulkRequest} - */ - public BulkRequest batchIngestionOption(String batchOption) { - this.batchIngestionOption = BatchIngestionOption.from(batchOption); - return this; - } - - /** - * Get batch ingestion option - * @return {@link BatchIngestionOption} - */ - public BatchIngestionOption batchIngestionOption() { - return this.batchIngestionOption; - } - /** * Set batch size * @param size batch size from input @@ -501,7 +481,6 @@ public void writeTo(StreamOutput out) throws IOException { refreshPolicy.writeTo(out); out.writeTimeValue(timeout); if (out.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { - out.writeEnum(batchIngestionOption); out.writeInt(batchSize); } } diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 3326c458b5752..ab8e823199447 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -39,7 +39,6 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.ResourceNotFoundException; import org.opensearch.action.DocWriteRequest; -import org.opensearch.action.bulk.BatchIngestionOption; import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.TransportBulkAction; import org.opensearch.action.index.IndexRequest; @@ -526,9 +525,8 @@ public void onFailure(Exception e) { @Override protected void doRun() { - BatchIngestionOption batchOption = originalBulkRequest.batchIngestionOption(); int batchSize = originalBulkRequest.batchSize(); - if (shouldExecuteBulkRequestInBatch(batchOption, originalBulkRequest.requests().size(), batchSize)) { + if (shouldExecuteBulkRequestInBatch(originalBulkRequest.requests().size(), batchSize)) { runBulkRequestInBatch(numberOfActionRequests, actionRequests, onFailure, onCompletion, onDropped, originalBulkRequest); return; } @@ -656,8 +654,8 @@ private void runBulkRequestInBatch( } } - private boolean shouldExecuteBulkRequestInBatch(BatchIngestionOption batchOption, int documentSize, int batchSize) { - return batchOption == BatchIngestionOption.ENABLED && documentSize > 1 && batchSize > 1; + private boolean shouldExecuteBulkRequestInBatch(int documentSize, int batchSize) { + return documentSize > 1 && batchSize > 1; } /** diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java index d090a22db3418..0bc4234c9b8b8 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java @@ -97,7 +97,6 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC Boolean defaultRequireAlias = request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, null); bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); bulkRequest.setRefreshPolicy(request.param("refresh")); - bulkRequest.batchIngestionOption(request.param("batch_ingestion_option", "")); bulkRequest.batchSize(request.paramAsInt("batch_size", 1)); bulkRequest.add( request.requiredContent(), diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 4608dc75742aa..6d216370bae9a 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -38,7 +38,6 @@ import org.opensearch.ResourceNotFoundException; import org.opensearch.Version; import org.opensearch.action.DocWriteRequest; -import org.opensearch.action.bulk.BatchIngestionOption; import org.opensearch.action.bulk.BulkRequest; import org.opensearch.action.bulk.TransportBulkAction; import org.opensearch.action.delete.DeleteRequest; @@ -143,7 +142,6 @@ public void setup() { when(threadPool.generic()).thenReturn(executorService); when(threadPool.executor(anyString())).thenReturn(executorService); mockBulkRequest = mock(BulkRequest.class); - lenient().when(mockBulkRequest.batchIngestionOption()).thenReturn(BatchIngestionOption.NONE); lenient().when(mockBulkRequest.batchSize()).thenReturn(1); } @@ -1710,7 +1708,6 @@ public void testExecuteBulkRequestInBatch() { bulkRequest.add(indexRequest3); IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest4); - bulkRequest.batchIngestionOption("enabled"); bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @@ -1747,7 +1744,6 @@ public void testExecuteBulkRequestInBatchWithDefaultAndFinalPipeline() { bulkRequest.add(indexRequest3); IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_final"); bulkRequest.add(indexRequest4); - bulkRequest.batchIngestionOption("enabled"); bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @@ -1777,7 +1773,6 @@ public void testExecuteBulkRequestInBatchFallbackWithOneDocument() { BulkRequest bulkRequest = new BulkRequest(); IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest1); - bulkRequest.batchIngestionOption("enabled"); bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @@ -1816,7 +1811,6 @@ public void testExecuteBulkRequestInBatchNoValidPipeline() { .setPipeline("_none") .setFinalPipeline("_none"); bulkRequest.add(indexRequest2); - bulkRequest.batchIngestionOption("enabled"); bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @@ -1847,7 +1841,6 @@ public void testExecuteBulkRequestInBatchNoValidDocument() { // will not be handled as not valid document type bulkRequest.add(new DeleteRequest("_index", "_id")); bulkRequest.add(new DeleteRequest("_index", "_id")); - bulkRequest.batchIngestionOption("enabled"); bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @@ -1881,7 +1874,6 @@ public void testExecuteBulkRequestInBatchWithException() { bulkRequest.add(indexRequest1); IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest2); - bulkRequest.batchIngestionOption("enabled"); bulkRequest.batchSize(2); @SuppressWarnings("unchecked") final BiConsumer failureHandler = mock(BiConsumer.class); @@ -1914,7 +1906,6 @@ public void testExecuteBulkRequestInBatchWithExceptionInCallback() { bulkRequest.add(indexRequest1); IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest2); - bulkRequest.batchIngestionOption("enabled"); bulkRequest.batchSize(2); List results = Arrays.asList( From 334e0c1b2083154b79d410633dd0e57efb4d9c33 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 30 Apr 2024 03:28:59 +0800 Subject: [PATCH 15/17] Throw invalid request exception for invalid batch_size Signed-off-by: Liyun Xiu --- .../rest-api-spec/test/ingest/70_bulk.yml | 15 +-------------- .../org/opensearch/action/bulk/BulkRequest.java | 7 +++---- 2 files changed, 4 insertions(+), 18 deletions(-) diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml index d319f68a75c95..edb7b77eb8d28 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml @@ -244,6 +244,7 @@ teardown: reason: "Added in 2.14.0" - do: + catch: bad_request bulk: refresh: true batch_size: -1 @@ -253,17 +254,3 @@ teardown: - '{"text": "text1"}' - '{"index": {"_index": "test_index", "_id": "test_id2"}}' - '{"text": "text2"}' - - - match: { errors: false } - - - do: - get: - index: test_index - id: test_id1 - - match: { _source: { "text": "text1", "field1": "value1" } } - - - do: - get: - index: test_index - id: test_id2 - - match: { _source: { "text": "text2", "field1": "value1" } } diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 11a38f679fb70..038b3edb53e67 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -357,11 +357,10 @@ public final BulkRequest timeout(TimeValue timeout) { * @return {@link BulkRequest} */ public BulkRequest batchSize(int size) { - if (size > 1) { - this.batchSize = size; - } else { - this.batchSize = 1; + if (size < 1) { + throw new IllegalArgumentException("batch_size must be larger than 0"); } + this.batchSize = size; return this; } From 68cabe16861a302a8f776eb722cc87e710884407 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 30 Apr 2024 07:57:08 +0800 Subject: [PATCH 16/17] Update server/src/main/java/org/opensearch/action/bulk/BulkRequest.java Co-authored-by: Andriy Redko Signed-off-by: Liyun Xiu --- .../src/main/java/org/opensearch/action/bulk/BulkRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 038b3edb53e67..c9343d8cf71fc 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -479,7 +479,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(requests, DocWriteRequest::writeDocumentRequest); refreshPolicy.writeTo(out); out.writeTimeValue(timeout); - if (out.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { + if (out.getVersion().onOrAfter(Version.V_2_14_0)) { out.writeInt(batchSize); } } From 3cc7f417363b9cffc3105cb0ece6d603ffba0798 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 30 Apr 2024 08:04:08 +0800 Subject: [PATCH 17/17] Remove version constant Signed-off-by: Liyun Xiu --- .../main/java/org/opensearch/action/bulk/BulkRequest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index c9343d8cf71fc..7614206cd226f 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -81,7 +81,6 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(BulkRequest.class); private static final int REQUEST_OVERHEAD = 50; - private static final Version MINIMAL_VERSION_SUPPORT_BATCH = Version.V_2_14_0; /** * Requests that are part of this request. It is only possible to add things that are both {@link ActionRequest}s and * {@link WriteRequest}s to this but java doesn't support syntax to declare that everything in the array has both types so we declare @@ -109,7 +108,7 @@ public BulkRequest(StreamInput in) throws IOException { requests.addAll(in.readList(i -> DocWriteRequest.readDocumentRequest(null, i))); refreshPolicy = RefreshPolicy.readFrom(in); timeout = in.readTimeValue(); - if (in.getVersion().onOrAfter(MINIMAL_VERSION_SUPPORT_BATCH)) { + if (in.getVersion().onOrAfter(Version.V_2_14_0)) { batchSize = in.readInt(); } } @@ -358,7 +357,7 @@ public final BulkRequest timeout(TimeValue timeout) { */ public BulkRequest batchSize(int size) { if (size < 1) { - throw new IllegalArgumentException("batch_size must be larger than 0"); + throw new IllegalArgumentException("batch_size must be greater than 0"); } this.batchSize = size; return this;