diff --git a/CHANGELOG.md b/CHANGELOG.md index e388364893345..93b4e4e263063 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -118,6 +118,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Mute the query profile IT with concurrent execution ([#9840](https://github.com/opensearch-project/OpenSearch/pull/9840)) - Force merge with `only_expunge_deletes` honors max segment size ([#10036](https://github.com/opensearch-project/OpenSearch/pull/10036)) - Add the means to extract the contextual properties from HttpChannel, TcpCChannel and TrasportChannel without excessive typecasting ([#10562](https://github.com/opensearch-project/OpenSearch/pull/10562)) +- Search pipelines now support asynchronous request and response processors to avoid blocking on a transport thread ([#10598](https://github.com/opensearch-project/OpenSearch/pull/10598)) - [Remote Store] Add Remote Store backpressure rejection stats to `_nodes/stats` ([#10524](https://github.com/opensearch-project/OpenSearch/pull/10524)) - [BUG] Fix java.lang.SecurityException in repository-gcs plugin ([#10642](https://github.com/opensearch-project/OpenSearch/pull/10642)) - Add telemetry tracer/metric enable flag and integ test. ([#10395](https://github.com/opensearch-project/OpenSearch/pull/10395)) diff --git a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java index a6fb8453af4ff..16b7e4810b130 100644 --- a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java @@ -506,24 +506,51 @@ private void executeRequest( ActionListener listener; try { searchRequest = searchPipelineService.resolvePipeline(originalSearchRequest); - listener = ActionListener.wrap( - r -> originalListener.onResponse(searchRequest.transformResponse(r)), - originalListener::onFailure - ); + listener = searchRequest.transformResponseListener(originalListener); } catch (Exception e) { originalListener.onFailure(e); return; } - if (searchQueryMetricsEnabled) { - try { - searchQueryCategorizer.categorize(searchRequest.source()); - } catch (Exception e) { - logger.error("Error while trying to categorize the query.", e); + ActionListener requestTransformListener = ActionListener.wrap(sr -> { + if (searchQueryMetricsEnabled) { + try { + searchQueryCategorizer.categorize(sr.source()); + } catch (Exception e) { + logger.error("Error while trying to categorize the query.", e); + } } - } - ActionListener rewriteListener = ActionListener.wrap(source -> { + ActionListener rewriteListener = buildRewriteListener( + sr, + task, + timeProvider, + searchAsyncActionProvider, + listener, + searchRequestOperationsListener + ); + if (sr.source() == null) { + rewriteListener.onResponse(sr.source()); + } else { + Rewriteable.rewriteAndFetch( + sr.source(), + searchService.getRewriteContext(timeProvider::getAbsoluteStartMillis), + rewriteListener + ); + } + }, listener::onFailure); + searchRequest.transformRequest(requestTransformListener); + } + + private ActionListener buildRewriteListener( + SearchRequest searchRequest, + Task task, + SearchTimeProvider timeProvider, + SearchAsyncActionProvider searchAsyncActionProvider, + ActionListener listener, + SearchRequestOperationsListener searchRequestOperationsListener + ) { + return ActionListener.wrap(source -> { if (source != searchRequest.source()) { // only set it if it changed - we don't allow null values to be set but it might be already null. this way we catch // situations when source is rewritten to null due to a bug @@ -634,15 +661,6 @@ private void executeRequest( } } }, listener::onFailure); - if (searchRequest.source() == null) { - rewriteListener.onResponse(searchRequest.source()); - } else { - Rewriteable.rewriteAndFetch( - searchRequest.source(), - searchService.getRewriteContext(timeProvider::getAbsoluteStartMillis), - rewriteListener - ); - } } static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) { diff --git a/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java b/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java index d4292b85b20a5..8bab961423f91 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java +++ b/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java @@ -16,11 +16,13 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.common.Nullable; import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.search.SearchPhaseResult; +import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -117,92 +119,138 @@ protected void afterResponseProcessor(Processor processor, long timeInNanos) {} protected void onResponseProcessorFailed(Processor processor) {} - SearchRequest transformRequest(SearchRequest request) throws SearchPipelineProcessingException { - if (searchRequestProcessors.isEmpty() == false) { - long pipelineStart = relativeTimeSupplier.getAsLong(); - beforeTransformRequest(); - try { - try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()) { - request.writeTo(bytesStreamOutput); - try (StreamInput in = bytesStreamOutput.bytes().streamInput()) { - try (StreamInput input = new NamedWriteableAwareStreamInput(in, namedWriteableRegistry)) { - request = new SearchRequest(input); - } - } - } - for (SearchRequestProcessor processor : searchRequestProcessors) { - beforeRequestProcessor(processor); - long start = relativeTimeSupplier.getAsLong(); - try { - request = processor.processRequest(request); - } catch (Exception e) { - onRequestProcessorFailed(processor); - if (processor.isIgnoreFailure()) { - logger.warn( - "The exception from request processor [" - + processor.getType() - + "] in the search pipeline [" - + id - + "] was ignored", - e - ); - } else { - throw e; - } - } finally { - long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); - afterRequestProcessor(processor, took); - } + void transformRequest(SearchRequest request, ActionListener requestListener) throws SearchPipelineProcessingException { + if (searchRequestProcessors.isEmpty()) { + requestListener.onResponse(request); + return; + } + + try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()) { + request.writeTo(bytesStreamOutput); + try (StreamInput in = bytesStreamOutput.bytes().streamInput()) { + try (StreamInput input = new NamedWriteableAwareStreamInput(in, namedWriteableRegistry)) { + request = new SearchRequest(input); } - } catch (Exception e) { - onTransformRequestFailure(); - throw new SearchPipelineProcessingException(e); - } finally { - long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart); - afterTransformRequest(took); } + } catch (IOException e) { + requestListener.onFailure(new SearchPipelineProcessingException(e)); + return; } - return request; + + ActionListener finalListener = getTerminalSearchRequestActionListener(requestListener); + + // Chain listeners back-to-front + ActionListener currentListener = finalListener; + for (int i = searchRequestProcessors.size() - 1; i >= 0; i--) { + final ActionListener nextListener = currentListener; + SearchRequestProcessor processor = searchRequestProcessors.get(i); + currentListener = ActionListener.wrap(r -> { + long start = relativeTimeSupplier.getAsLong(); + beforeRequestProcessor(processor); + processor.processRequestAsync(r, ActionListener.wrap(rr -> { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); + afterRequestProcessor(processor, took); + nextListener.onResponse(rr); + }, e -> { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); + afterRequestProcessor(processor, took); + onRequestProcessorFailed(processor); + if (processor.isIgnoreFailure()) { + logger.warn( + "The exception from request processor [" + + processor.getType() + + "] in the search pipeline [" + + id + + "] was ignored", + e + ); + nextListener.onResponse(r); + } else { + nextListener.onFailure(new SearchPipelineProcessingException(e)); + } + })); + }, finalListener::onFailure); + } + + beforeTransformRequest(); + currentListener.onResponse(request); } - SearchResponse transformResponse(SearchRequest request, SearchResponse response) throws SearchPipelineProcessingException { - if (searchResponseProcessors.isEmpty() == false) { - long pipelineStart = relativeTimeSupplier.getAsLong(); - beforeTransformResponse(); - try { - for (SearchResponseProcessor processor : searchResponseProcessors) { - beforeResponseProcessor(processor); - long start = relativeTimeSupplier.getAsLong(); - try { - response = processor.processResponse(request, response); - } catch (Exception e) { - onResponseProcessorFailed(processor); - if (processor.isIgnoreFailure()) { - logger.warn( - "The exception from response processor [" - + processor.getType() - + "] in the search pipeline [" - + id - + "] was ignored", - e - ); - } else { - throw e; - } - } finally { - long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); - afterResponseProcessor(processor, took); + private ActionListener getTerminalSearchRequestActionListener(ActionListener requestListener) { + final long pipelineStart = relativeTimeSupplier.getAsLong(); + + return ActionListener.wrap(r -> { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart); + afterTransformRequest(took); + requestListener.onResponse(new PipelinedRequest(this, r)); + }, e -> { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart); + afterTransformRequest(took); + onTransformRequestFailure(); + requestListener.onFailure(new SearchPipelineProcessingException(e)); + }); + } + + ActionListener transformResponseListener(SearchRequest request, ActionListener responseListener) { + if (searchResponseProcessors.isEmpty()) { + // No response transformation necessary + return responseListener; + } + + long[] pipelineStart = new long[1]; + + final ActionListener originalListener = responseListener; + responseListener = ActionListener.wrap(r -> { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart[0]); + afterTransformResponse(took); + originalListener.onResponse(r); + }, e -> { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart[0]); + afterTransformResponse(took); + onTransformResponseFailure(); + originalListener.onFailure(e); + }); + ActionListener finalListener = responseListener; // Jump directly to this one on exception. + + for (int i = searchResponseProcessors.size() - 1; i >= 0; i--) { + final ActionListener currentFinalListener = responseListener; + final SearchResponseProcessor processor = searchResponseProcessors.get(i); + + responseListener = ActionListener.wrap(r -> { + beforeResponseProcessor(processor); + final long start = relativeTimeSupplier.getAsLong(); + processor.processResponseAsync(request, r, ActionListener.wrap(rr -> { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); + afterResponseProcessor(processor, took); + currentFinalListener.onResponse(rr); + }, e -> { + onResponseProcessorFailed(processor); + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); + afterResponseProcessor(processor, took); + if (processor.isIgnoreFailure()) { + logger.warn( + "The exception from response processor [" + + processor.getType() + + "] in the search pipeline [" + + id + + "] was ignored", + e + ); + // Pass the previous response through to the next processor in the chain + currentFinalListener.onResponse(r); + } else { + currentFinalListener.onFailure(new SearchPipelineProcessingException(e)); } - } - } catch (Exception e) { - onTransformResponseFailure(); - throw new SearchPipelineProcessingException(e); - } finally { - long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart); - afterTransformResponse(took); - } + })); + }, finalListener::onFailure); } - return response; + final ActionListener chainListener = responseListener; + return ActionListener.wrap(r -> { + beforeTransformResponse(); + pipelineStart[0] = relativeTimeSupplier.getAsLong(); + chainListener.onResponse(r); + }, originalListener::onFailure); + } void runSearchPhaseResultsTransformer( diff --git a/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java b/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java index 5a7539808c127..77dfc6bcd4fc5 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java +++ b/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java @@ -12,6 +12,7 @@ import org.opensearch.action.search.SearchPhaseResults; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; +import org.opensearch.core.action.ActionListener; import org.opensearch.search.SearchPhaseResult; /** @@ -27,8 +28,12 @@ public final class PipelinedRequest extends SearchRequest { this.pipeline = pipeline; } - public SearchResponse transformResponse(SearchResponse response) { - return pipeline.transformResponse(this, response); + public void transformRequest(ActionListener requestListener) { + pipeline.transformRequest(this, requestListener); + } + + public ActionListener transformResponseListener(ActionListener responseListener) { + return pipeline.transformResponseListener(this, responseListener); } public void transformSearchPhaseResults( diff --git a/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java index 739101519ff98..580fe1b7c4216 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java +++ b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java @@ -408,8 +408,7 @@ public PipelinedRequest resolvePipeline(SearchRequest searchRequest) { pipeline = pipelineHolder.pipeline; } } - SearchRequest transformedRequest = pipeline.transformRequest(searchRequest); - return new PipelinedRequest(pipeline, transformedRequest); + return new PipelinedRequest(pipeline, searchRequest); } Map> getRequestProcessorFactories() { diff --git a/server/src/main/java/org/opensearch/search/pipeline/SearchRequestProcessor.java b/server/src/main/java/org/opensearch/search/pipeline/SearchRequestProcessor.java index c236cde1a5cc0..427c9e4ab694c 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/SearchRequestProcessor.java +++ b/server/src/main/java/org/opensearch/search/pipeline/SearchRequestProcessor.java @@ -9,10 +9,37 @@ package org.opensearch.search.pipeline; import org.opensearch.action.search.SearchRequest; +import org.opensearch.core.action.ActionListener; /** * Interface for a search pipeline processor that modifies a search request. */ public interface SearchRequestProcessor extends Processor { + + /** + * Transform a {@link SearchRequest}. Executed on the coordinator node before any {@link org.opensearch.action.search.SearchPhase} + * executes. + *

+ * Implement this method if the processor makes no asynchronous calls. + * @param request the executed {@link SearchRequest} + * @return a new {@link SearchRequest} (or the input {@link SearchRequest} if no changes) + * @throws Exception if an error occurs during processing + */ SearchRequest processRequest(SearchRequest request) throws Exception; + + /** + * Transform a {@link SearchRequest}. Executed on the coordinator node before any {@link org.opensearch.action.search.SearchPhase} + * executes. + *

+ * Expert method: Implement this if the processor needs to make asynchronous calls. Otherwise, implement processRequest. + * @param request the executed {@link SearchRequest} + * @param requestListener callback to be invoked on successful processing or on failure + */ + default void processRequestAsync(SearchRequest request, ActionListener requestListener) { + try { + requestListener.onResponse(processRequest(request)); + } catch (Exception e) { + requestListener.onFailure(e); + } + } } diff --git a/server/src/main/java/org/opensearch/search/pipeline/SearchResponseProcessor.java b/server/src/main/java/org/opensearch/search/pipeline/SearchResponseProcessor.java index 2f22cedb9b5c0..21136ce208fee 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/SearchResponseProcessor.java +++ b/server/src/main/java/org/opensearch/search/pipeline/SearchResponseProcessor.java @@ -10,10 +10,37 @@ import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; +import org.opensearch.core.action.ActionListener; /** * Interface for a search pipeline processor that modifies a search response. */ public interface SearchResponseProcessor extends Processor { + + /** + * Transform a {@link SearchResponse}, possibly based on the executed {@link SearchRequest}. + *

+ * Implement this method if the processor makes no asynchronous calls. + * @param request the executed {@link SearchRequest} + * @param response the current {@link SearchResponse}, possibly modified by earlier processors + * @return a modified {@link SearchResponse} (or the input {@link SearchResponse} if no changes) + * @throws Exception if an error occurs during processing + */ SearchResponse processResponse(SearchRequest request, SearchResponse response) throws Exception; + + /** + * Transform a {@link SearchResponse}, possibly based on the executed {@link SearchRequest}. + *

+ * Expert method: Implement this if the processor needs to make asynchronous calls. Otherwise, implement processResponse. + * @param request the executed {@link SearchRequest} + * @param response the current {@link SearchResponse}, possibly modified by earlier processors + * @param responseListener callback to be invoked on successful processing or on failure + */ + default void processResponseAsync(SearchRequest request, SearchResponse response, ActionListener responseListener) { + try { + responseListener.onResponse(processResponse(request, response)); + } catch (Exception e) { + responseListener.onFailure(e); + } + } } diff --git a/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java index d44bd3831281f..98d2a7e84d672 100644 --- a/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java +++ b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java @@ -41,6 +41,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.OpenSearchExecutors; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.breaker.CircuitBreaker; import org.opensearch.core.common.breaker.NoopCircuitBreaker; import org.opensearch.core.common.bytes.BytesArray; @@ -194,7 +195,7 @@ public void testResolveIndexDefaultPipeline() throws Exception { service.applyClusterState(cce); SearchRequest searchRequest = new SearchRequest("my_index").source(SearchSourceBuilder.searchSource().size(5)); - PipelinedRequest pipelinedRequest = service.resolvePipeline(searchRequest); + PipelinedRequest pipelinedRequest = syncTransformRequest(service.resolvePipeline(searchRequest)); assertEquals("p1", pipelinedRequest.getPipeline().getId()); assertEquals(10, pipelinedRequest.source().size()); @@ -597,7 +598,7 @@ public void testTransformRequest() throws Exception { SearchSourceBuilder sourceBuilder = new SearchSourceBuilder().query(new TermQueryBuilder("foo", "bar")).size(size); SearchRequest request = new SearchRequest("_index").source(sourceBuilder).pipeline("p1"); - PipelinedRequest pipelinedRequest = searchPipelineService.resolvePipeline(request); + PipelinedRequest pipelinedRequest = syncTransformRequest(searchPipelineService.resolvePipeline(request)); assertEquals(2 * size, pipelinedRequest.source().size()); assertEquals(size, request.source().size()); @@ -641,19 +642,57 @@ public void testTransformResponse() throws Exception { // First try without specifying a pipeline, which should be a no-op. SearchRequest searchRequest = new SearchRequest(); PipelinedRequest pipelinedRequest = searchPipelineService.resolvePipeline(searchRequest); - SearchResponse notTransformedResponse = pipelinedRequest.transformResponse(searchResponse); + SearchResponse notTransformedResponse = syncTransformResponse(pipelinedRequest, searchResponse); assertSame(searchResponse, notTransformedResponse); // Now apply a pipeline searchRequest = new SearchRequest().pipeline("p1"); pipelinedRequest = searchPipelineService.resolvePipeline(searchRequest); - SearchResponse transformedResponse = pipelinedRequest.transformResponse(searchResponse); + SearchResponse transformedResponse = syncTransformResponse(pipelinedRequest, searchResponse); assertEquals(size, transformedResponse.getHits().getHits().length); for (int i = 0; i < size; i++) { assertEquals(2.0, transformedResponse.getHits().getHits()[i].getScore(), 0.0001f); } } + /** + * Helper to synchronously apply a response pipeline, returning the transformed response. + */ + private static SearchResponse syncTransformResponse(PipelinedRequest pipelinedRequest, SearchResponse searchResponse) throws Exception { + SearchResponse[] responseBox = new SearchResponse[1]; + Exception[] exceptionBox = new Exception[1]; + ActionListener responseListener = pipelinedRequest.transformResponseListener(ActionListener.wrap(r -> { + responseBox[0] = r; + }, e -> { exceptionBox[0] = e; })); + responseListener.onResponse(searchResponse); + + if (exceptionBox[0] != null) { + throw exceptionBox[0]; + } + return responseBox[0]; + } + + /** + * Helper to synchronously apply a request pipeline, returning the transformed request. + */ + private static PipelinedRequest syncTransformRequest(PipelinedRequest request) throws Exception { + PipelinedRequest[] requestBox = new PipelinedRequest[1]; + Exception[] exceptionBox = new Exception[1]; + + request.transformRequest(ActionListener.wrap(r -> requestBox[0] = (PipelinedRequest) r, e -> exceptionBox[0] = e)); + if (exceptionBox[0] != null) { + throw exceptionBox[0]; + } + return requestBox[0]; + } + + /** + * Helper to synchronously apply a request pipeline and response pipeline, returning the transformed response. + */ + private static SearchResponse syncExecutePipeline(PipelinedRequest request, SearchResponse response) throws Exception { + return syncTransformResponse(syncTransformRequest(request), response); + } + public void testTransformSearchPhase() { SearchPipelineService searchPipelineService = createWithProcessors(); SearchPipelineMetadata metadata = new SearchPipelineMetadata( @@ -875,7 +914,7 @@ public void testInlinePipeline() throws Exception { SearchRequest searchRequest = new SearchRequest().source(sourceBuilder); // Verify pipeline - PipelinedRequest pipelinedRequest = searchPipelineService.resolvePipeline(searchRequest); + PipelinedRequest pipelinedRequest = syncTransformRequest(searchPipelineService.resolvePipeline(searchRequest)); Pipeline pipeline = pipelinedRequest.getPipeline(); assertEquals(SearchPipelineService.AD_HOC_PIPELINE_ID, pipeline.getId()); assertEquals(1, pipeline.getSearchRequestProcessors().size()); @@ -894,7 +933,7 @@ public void testInlinePipeline() throws Exception { SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); SearchResponse searchResponse = new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); - SearchResponse transformedResponse = pipeline.transformResponse(searchRequest, searchResponse); + SearchResponse transformedResponse = syncTransformResponse(pipelinedRequest, searchResponse); for (int i = 0; i < size; i++) { assertEquals(2.0, transformedResponse.getHits().getHits()[i].getScore(), 0.0001); } @@ -946,7 +985,10 @@ public void testExceptionOnRequestProcessing() { SearchRequest searchRequest = new SearchRequest().source(sourceBuilder); // Exception thrown when processing the request - expectThrows(SearchPipelineProcessingException.class, () -> searchPipelineService.resolvePipeline(searchRequest)); + expectThrows( + SearchPipelineProcessingException.class, + () -> syncTransformRequest(searchPipelineService.resolvePipeline(searchRequest)) + ); } public void testExceptionOnResponseProcessing() throws Exception { @@ -974,10 +1016,10 @@ public void testExceptionOnResponseProcessing() throws Exception { SearchResponse response = new SearchResponse(null, null, 0, 0, 0, 0, null, null); // Exception thrown when processing response - expectThrows(SearchPipelineProcessingException.class, () -> pipelinedRequest.transformResponse(response)); + expectThrows(SearchPipelineProcessingException.class, () -> syncTransformResponse(pipelinedRequest, response)); } - public void testCatchExceptionOnRequestProcessing() throws IllegalAccessException { + public void testCatchExceptionOnRequestProcessing() throws Exception { SearchRequestProcessor throwingRequestProcessor = new FakeRequestProcessor("throwing_request", null, null, true, r -> { throw new RuntimeException(); }); @@ -1008,7 +1050,7 @@ public void testCatchExceptionOnRequestProcessing() throws IllegalAccessExceptio "The exception from request processor [throwing_request] in the search pipeline [_ad_hoc_pipeline] was ignored" ) ); - PipelinedRequest pipelinedRequest = searchPipelineService.resolvePipeline(searchRequest); + syncTransformRequest(searchPipelineService.resolvePipeline(searchRequest)); mockAppender.assertAllExpectationsMatched(); } } @@ -1048,7 +1090,7 @@ public void testCatchExceptionOnResponseProcessing() throws Exception { "The exception from response processor [throwing_response] in the search pipeline [_ad_hoc_pipeline] was ignored" ) ); - pipelinedRequest.transformResponse(response); + syncTransformResponse(pipelinedRequest, response); mockAppender.assertAllExpectationsMatched(); } } @@ -1078,15 +1120,15 @@ public void testStats() throws Exception { SearchRequest request = new SearchRequest(); SearchResponse response = new SearchResponse(null, null, 0, 0, 0, 0, null, null); - searchPipelineService.resolvePipeline(request.pipeline("good_request_pipeline")).transformResponse(response); + syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("good_request_pipeline")), response); expectThrows( SearchPipelineProcessingException.class, - () -> searchPipelineService.resolvePipeline(request.pipeline("bad_request_pipeline")).transformResponse(response) + () -> syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("bad_request_pipeline")), response) ); - searchPipelineService.resolvePipeline(request.pipeline("good_response_pipeline")).transformResponse(response); + syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("good_response_pipeline")), response); expectThrows( SearchPipelineProcessingException.class, - () -> searchPipelineService.resolvePipeline(request.pipeline("bad_response_pipeline")).transformResponse(response) + () -> syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("bad_response_pipeline")), response) ); SearchPipelineStats stats = searchPipelineService.stats(); @@ -1164,12 +1206,12 @@ public void testStatsEnabledIgnoreFailure() throws Exception { SearchRequest request = new SearchRequest(); SearchResponse response = new SearchResponse(null, null, 0, 0, 0, 0, null, null); - searchPipelineService.resolvePipeline(request.pipeline("good_request_pipeline")).transformResponse(response); + syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("good_request_pipeline")), response); // Caught Exception here - searchPipelineService.resolvePipeline(request.pipeline("bad_request_pipeline")).transformResponse(response); - searchPipelineService.resolvePipeline(request.pipeline("good_response_pipeline")).transformResponse(response); + syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("bad_request_pipeline")), response); + syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("good_response_pipeline")), response); // Caught Exception here - searchPipelineService.resolvePipeline(request.pipeline("bad_response_pipeline")).transformResponse(response); + syncExecutePipeline(searchPipelineService.resolvePipeline(request.pipeline("bad_response_pipeline")), response); // when ignoreFailure enabled, the search pipelines will all succeed. SearchPipelineStats stats = searchPipelineService.stats(); @@ -1273,8 +1315,8 @@ private SearchPipelineService getSearchPipelineService( } private static void assertPipelineStats(OperationStats stats, long count, long failed) { - assertEquals(stats.getCount(), count); - assertEquals(stats.getFailedCount(), failed); + assertEquals(count, stats.getCount()); + assertEquals(failed, stats.getFailedCount()); } public void testAdHocRejectingProcessor() {