From 5d16d35ab5968c660fab545ee83aec551a92e547 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 13 Aug 2019 16:48:06 +0200 Subject: [PATCH 1/5] Add support for cancelling async requests in low-level REST client (#45379) The low-level REST client exposes a `performRequestAsync` method that allows to send async requests, but today it does not expose the ability to cancel such requests. That is something that the underlying apache async http client supports, and it makes sense for us to expose. This commit adds a return value to the `performRequestAsync` method, which is backwards compatible. A `Cancellable` object gets returned, which exposes a `cancel` public method. When calling `cancel`, the on-going request associated with the returned `Cancellable` instance will be cancelled by calling its `abort` method. This works throughout multiple retries, though some special care was needed for the case where `cancel` is called between different attempts (when one attempt has failed and the consecutive one has not been sent yet). Note that cancelling a request on the client side does not automatically translate to cancelling the server side execution of it. That needs to be specifically implemented, which is on the work for the search API (see #43332). Relates to #44802 --- .../org/elasticsearch/client/Cancellable.java | 87 +++++++++++++++ .../org/elasticsearch/client/RestClient.java | 77 +++++++------ .../RestClientMultipleHostsIntegTests.java | 78 +++++++++++-- .../client/RestClientMultipleHostsTests.java | 19 ++-- .../RestClientSingleHostIntegTests.java | 103 +++++++++++++++++- .../client/RestClientSingleHostTests.java | 30 ++--- .../RestClientDocumentation.java | 40 +++++-- docs/java-rest/low-level/usage.asciidoc | 18 ++- 8 files changed, 366 insertions(+), 86 deletions(-) create mode 100644 client/rest/src/main/java/org/elasticsearch/client/Cancellable.java diff --git a/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java b/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java new file mode 100644 index 0000000000000..6a31ab3fe1744 --- /dev/null +++ b/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java @@ -0,0 +1,87 @@ +/* + * 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. + */ +package org.elasticsearch.client; + +import org.apache.http.client.methods.AbstractExecutionAwareRequest; +import org.apache.http.client.methods.HttpRequestBase; + +import java.util.concurrent.CancellationException; + +/** + * Represents an operation that can be cancelled. + * Returned when executing async requests through {@link RestClient#performRequestAsync(Request, ResponseListener)}, so that the request + * can be cancelled if needed. Cancelling a request will result in calling {@link AbstractExecutionAwareRequest#abort()} on the underlying + * request object, which will in turn cancel its corresponding {@link java.util.concurrent.Future}. + * Note that cancelling a request does not automatically translate to aborting its execution on the server side, which needs to be + * specifically implemented in each API. + */ +public class Cancellable { + + static final Cancellable NO_OP = new Cancellable(null) { + @Override + public void cancel() { + } + + @Override + void runIfNotCancelled(Runnable runnable) { + throw new UnsupportedOperationException(); + } + }; + + static Cancellable fromRequest(HttpRequestBase httpRequest) { + return new Cancellable(httpRequest); + } + + private final HttpRequestBase httpRequest; + + private Cancellable(HttpRequestBase httpRequest) { + this.httpRequest = httpRequest; + } + + /** + * Cancels the on-going request that is associated with the current instance of {@link Cancellable}. + * + */ + public synchronized void cancel() { + this.httpRequest.abort(); + } + + /** + * Executes some arbitrary code iff the on-going request has not been cancelled, otherwise throws {@link CancellationException}. + * This is needed to guarantee that cancelling a request works correctly even in case {@link #cancel()} is called between different + * attempts of the same request. The low-level client reuses the same instance of the {@link AbstractExecutionAwareRequest} by calling + * {@link AbstractExecutionAwareRequest#reset()} between subsequent retries. The {@link #cancel()} method can be called at anytime, + * and we need to handle the case where it gets called while there is no request being executed as one attempt may have failed and + * the subsequent attempt has not been started yet. + * If the request has already been cancelled we don't go ahead with the next attempt, and artificially raise the + * {@link CancellationException}, otherwise we run the provided {@link Runnable} which will reset the request and send the next attempt. + * Note that this method must be synchronized as well as the {@link #cancel()} method, to prevent a request from being cancelled + * when there is no future to cancel, which would make cancelling the request a no-op. + */ + synchronized void runIfNotCancelled(Runnable runnable) { + if (this.httpRequest.isAborted()) { + throw newCancellationException(); + } + runnable.run(); + } + + static CancellationException newCancellationException() { + return new CancellationException("request was cancelled"); + } +} diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index 38185ac960926..a31732d742731 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -277,60 +277,64 @@ private ResponseOrResponseException convertResponse(InternalRequest request, Nod * @param responseListener the {@link ResponseListener} to notify when the * request is completed or fails */ - public void performRequestAsync(Request request, ResponseListener responseListener) { + public Cancellable performRequestAsync(Request request, ResponseListener responseListener) { try { FailureTrackingResponseListener failureTrackingResponseListener = new FailureTrackingResponseListener(responseListener); InternalRequest internalRequest = new InternalRequest(request); performRequestAsync(nextNodes(), internalRequest, failureTrackingResponseListener); + return internalRequest.cancellable; } catch (Exception e) { responseListener.onFailure(e); + return Cancellable.NO_OP; } } private void performRequestAsync(final NodeTuple> nodeTuple, final InternalRequest request, final FailureTrackingResponseListener listener) { - final RequestContext context = request.createContextForNextAttempt(nodeTuple.nodes.next(), nodeTuple.authCache); - client.execute(context.requestProducer, context.asyncResponseConsumer, context.context, new FutureCallback() { - @Override - public void completed(HttpResponse httpResponse) { - try { - ResponseOrResponseException responseOrResponseException = convertResponse(request, context.node, httpResponse); - if (responseOrResponseException.responseException == null) { - listener.onSuccess(responseOrResponseException.response); - } else { - if (nodeTuple.nodes.hasNext()) { - listener.trackFailure(responseOrResponseException.responseException); - performRequestAsync(nodeTuple, request, listener); + request.cancellable.runIfNotCancelled(() -> { + final RequestContext context = request.createContextForNextAttempt(nodeTuple.nodes.next(), nodeTuple.authCache); + client.execute(context.requestProducer, context.asyncResponseConsumer, context.context, new FutureCallback() { + @Override + public void completed(HttpResponse httpResponse) { + try { + ResponseOrResponseException responseOrResponseException = convertResponse(request, context.node, httpResponse); + if (responseOrResponseException.responseException == null) { + listener.onSuccess(responseOrResponseException.response); } else { - listener.onDefinitiveFailure(responseOrResponseException.responseException); + if (nodeTuple.nodes.hasNext()) { + listener.trackFailure(responseOrResponseException.responseException); + performRequestAsync(nodeTuple, request, listener); + } else { + listener.onDefinitiveFailure(responseOrResponseException.responseException); + } } + } catch(Exception e) { + listener.onDefinitiveFailure(e); } - } catch(Exception e) { - listener.onDefinitiveFailure(e); } - } - @Override - public void failed(Exception failure) { - try { - RequestLogger.logFailedRequest(logger, request.httpRequest, context.node, failure); - onFailure(context.node); - if (nodeTuple.nodes.hasNext()) { - listener.trackFailure(failure); - performRequestAsync(nodeTuple, request, listener); - } else { - listener.onDefinitiveFailure(failure); + @Override + public void failed(Exception failure) { + try { + RequestLogger.logFailedRequest(logger, request.httpRequest, context.node, failure); + onFailure(context.node); + if (nodeTuple.nodes.hasNext()) { + listener.trackFailure(failure); + performRequestAsync(nodeTuple, request, listener); + } else { + listener.onDefinitiveFailure(failure); + } + } catch(Exception e) { + listener.onDefinitiveFailure(e); } - } catch(Exception e) { - listener.onDefinitiveFailure(e); } - } - @Override - public void cancelled() { - listener.onDefinitiveFailure(new ExecutionException("request was cancelled", null)); - } + @Override + public void cancelled() { + listener.onDefinitiveFailure(Cancellable.newCancellationException()); + } + }); }); } @@ -651,19 +655,20 @@ public void remove() { private class InternalRequest { private final Request request; - private final Map params; private final Set ignoreErrorCodes; private final HttpRequestBase httpRequest; + private final Cancellable cancellable; private final WarningsHandler warningsHandler; InternalRequest(Request request) { this.request = request; - this.params = new HashMap<>(request.getParameters()); + Map params = new HashMap<>(request.getParameters()); //ignore is a special parameter supported by the clients, shouldn't be sent to es String ignoreString = params.remove("ignore"); this.ignoreErrorCodes = getIgnoreErrorCodes(ignoreString, request.getMethod()); URI uri = buildUri(pathPrefix, request.getEndpoint(), params); this.httpRequest = createHttpRequest(request.getMethod(), uri, request.getEntity()); + this.cancellable = Cancellable.fromRequest(httpRequest); setHeaders(httpRequest, request.getOptions().getHeaders()); this.warningsHandler = request.getOptions().getWarningsHandler() == null ? RestClient.this.warningsHandler : request.getOptions().getWarningsHandler(); diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java index 4cc16c45bab2f..38317af790ae7 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.concurrent.CancellationException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -42,7 +43,9 @@ import static org.elasticsearch.client.RestClientTestUtil.getAllStatusCodes; import static org.elasticsearch.client.RestClientTestUtil.randomErrorNoRetryStatusCode; import static org.elasticsearch.client.RestClientTestUtil.randomOkStatusCode; +import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -52,6 +55,7 @@ */ public class RestClientMultipleHostsIntegTests extends RestClientTestCase { + private static WaitForCancelHandler waitForCancelHandler; private static HttpServer[] httpServers; private static HttpHost[] httpHosts; private static boolean stoppedFirstHost = false; @@ -94,9 +98,34 @@ private static HttpServer createHttpServer() throws Exception { for (int statusCode : getAllStatusCodes()) { httpServer.createContext(pathPrefix + "/" + statusCode, new ResponseHandler(statusCode)); } + waitForCancelHandler = new WaitForCancelHandler(); + httpServer.createContext(pathPrefix + "/wait", waitForCancelHandler); return httpServer; } + private static class WaitForCancelHandler implements HttpHandler { + private CountDownLatch cancelHandlerLatch; + + void reset() { + cancelHandlerLatch = new CountDownLatch(1); + } + + void cancelDone() { + cancelHandlerLatch.countDown(); + } + + @Override + public void handle(HttpExchange exchange) throws IOException { + try { + cancelHandlerLatch.await(); + } catch (InterruptedException ignore) { + } finally { + exchange.sendResponseHeaders(200, 0); + exchange.close(); + } + } + } + private static class ResponseHandler implements HttpHandler { private final int statusCode; @@ -127,7 +156,7 @@ public void stopRandomHost() { //verify that shutting down some hosts doesn't matter as long as one working host is left behind if (httpServers.length > 1 && randomBoolean()) { List updatedHttpServers = new ArrayList<>(httpServers.length - 1); - int nodeIndex = randomInt(httpServers.length - 1); + int nodeIndex = randomIntBetween(0, httpServers.length - 1); if (0 == nodeIndex) { stoppedFirstHost = true; } @@ -139,7 +168,7 @@ public void stopRandomHost() { updatedHttpServers.add(httpServer); } } - httpServers = updatedHttpServers.toArray(new HttpServer[updatedHttpServers.size()]); + httpServers = updatedHttpServers.toArray(new HttpServer[0]); } } @@ -195,6 +224,40 @@ public void onFailure(Exception exception) { } } + public void testCancelAsyncRequests() throws Exception { + int numRequests = randomIntBetween(5, 20); + final CountDownLatch latch = new CountDownLatch(numRequests); + final List responses = new CopyOnWriteArrayList<>(); + final List exceptions = new CopyOnWriteArrayList<>(); + for (int i = 0; i < numRequests; i++) { + waitForCancelHandler.reset(); + final String method = RestClientTestUtil.randomHttpMethod(getRandom()); + //we don't test status codes that are subject to retries as they interfere with hosts being stopped + final int statusCode = randomBoolean() ? randomOkStatusCode(getRandom()) : randomErrorNoRetryStatusCode(getRandom()); + Cancellable cancellable = restClient.performRequestAsync(new Request(method, "/" + statusCode), new ResponseListener() { + @Override + public void onSuccess(Response response) { + responses.add(response); + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + exceptions.add(exception); + latch.countDown(); + } + }); + cancellable.cancel(); + waitForCancelHandler.cancelDone(); + } + assertTrue(latch.await(5, TimeUnit.SECONDS)); + assertEquals(0, responses.size()); + assertEquals(numRequests, exceptions.size()); + for (Exception exception : exceptions) { + assertThat(exception, instanceOf(CancellationException.class)); + } + } + /** * Test host selector against a real server and * test what happens after calling @@ -249,13 +312,10 @@ Response getResponse() { } private NodeSelector firstPositionNodeSelector() { - return new NodeSelector() { - @Override - public void select(Iterable nodes) { - for (Iterator itr = nodes.iterator(); itr.hasNext();) { - if (httpHosts[0] != itr.next().getHost()) { - itr.remove(); - } + return nodes -> { + for (Iterator itr = nodes.iterator(); itr.hasNext();) { + if (httpHosts[0] != itr.next().getHost()) { + itr.remove(); } } }; diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java index f3df9bf3bfd37..21d4e9d0e81fe 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsTests.java @@ -243,19 +243,16 @@ public void testRoundRobinRetryErrors() throws Exception { } public void testNodeSelector() throws Exception { - NodeSelector firstPositionOnly = new NodeSelector() { - @Override - public void select(Iterable restClientNodes) { - boolean found = false; - for (Iterator itr = restClientNodes.iterator(); itr.hasNext();) { - if (nodes.get(0) == itr.next()) { - found = true; - } else { - itr.remove(); - } + NodeSelector firstPositionOnly = restClientNodes -> { + boolean found = false; + for (Iterator itr = restClientNodes.iterator(); itr.hasNext();) { + if (nodes.get(0) == itr.next()) { + found = true; + } else { + itr.remove(); } - assertTrue(found); } + assertTrue(found); }; RestClient restClient = createRestClient(firstPositionOnly); int rounds = between(1, 10); diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java index e3fd3c311378b..c47fa55c9fe67 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java @@ -26,11 +26,15 @@ import org.apache.http.Consts; import org.apache.http.Header; import org.apache.http.HttpHost; +import org.apache.http.HttpResponse; import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpRequestBase; import org.apache.http.entity.ContentType; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.client.TargetAuthenticationStrategy; +import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.http.message.BasicHeader; import org.apache.http.nio.entity.NStringEntity; @@ -49,16 +53,22 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CancellationException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import static org.elasticsearch.client.RestClientTestUtil.getAllStatusCodes; import static org.elasticsearch.client.RestClientTestUtil.getHttpMethods; +import static org.elasticsearch.client.RestClientTestUtil.randomHttpMethod; import static org.elasticsearch.client.RestClientTestUtil.randomStatusCode; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -73,6 +83,7 @@ public class RestClientSingleHostIntegTests extends RestClientTestCase { private RestClient restClient; private String pathPrefix; private Header[] defaultHeaders; + private WaitForCancelHandler waitForCancelHandler; @Before public void startHttpServer() throws Exception { @@ -89,9 +100,31 @@ private HttpServer createHttpServer() throws Exception { for (int statusCode : getAllStatusCodes()) { httpServer.createContext(pathPrefix + "/" + statusCode, new ResponseHandler(statusCode)); } + waitForCancelHandler = new WaitForCancelHandler(); + httpServer.createContext(pathPrefix + "/wait", waitForCancelHandler); return httpServer; } + private class WaitForCancelHandler implements HttpHandler { + + private final CountDownLatch cancelHandlerLatch = new CountDownLatch(1); + + void cancelDone() { + cancelHandlerLatch.countDown(); + } + + @Override + public void handle(HttpExchange exchange) throws IOException { + try { + cancelHandlerLatch.await(); + } catch (InterruptedException ignore) { + } finally { + exchange.sendResponseHeaders(200, 0); + exchange.close(); + } + } + } + private static class ResponseHandler implements HttpHandler { private final int statusCode; @@ -201,6 +234,75 @@ public void onFailure(Exception exception) { } } + public void testCancelAsyncRequest() throws Exception { + Request request = new Request(randomHttpMethod(getRandom()), "/wait"); + CountDownLatch requestLatch = new CountDownLatch(1); + AtomicReference error = new AtomicReference<>(); + Cancellable cancellable = restClient.performRequestAsync(request, new ResponseListener() { + @Override + public void onSuccess(Response response) { + throw new AssertionError("onResponse called unexpectedly"); + } + + @Override + public void onFailure(Exception exception) { + error.set(exception); + requestLatch.countDown(); + } + }); + cancellable.cancel(); + waitForCancelHandler.cancelDone(); + assertTrue(requestLatch.await(5, TimeUnit.SECONDS)); + assertThat(error.get(), instanceOf(CancellationException.class)); + } + + /** + * This test verifies some assumptions that we rely upon around the way the async http client works when reusing the same request + * throughout multiple retries, and the use of the {@link HttpRequestBase#abort()} method. + */ + public void testRequestResetAndAbort() throws Exception { + try (CloseableHttpAsyncClient client = HttpAsyncClientBuilder.create().build()) { + client.start(); + HttpHost httpHost = new HttpHost(httpServer.getAddress().getHostString(), httpServer.getAddress().getPort()); + HttpGet httpGet = new HttpGet(pathPrefix + "/200"); + + //calling abort before the request is sent is a no-op + httpGet.abort(); + assertTrue(httpGet.isAborted()); + + { + httpGet.reset(); + assertFalse(httpGet.isAborted()); + httpGet.abort();//this has no effect on the next call (although isAborted will return true until the next reset) + Future future = client.execute(httpHost, httpGet, null); + assertEquals(200, future.get().getStatusLine().getStatusCode()); + assertFalse(future.isCancelled()); + } + { + httpGet.reset(); + Future future = client.execute(httpHost, httpGet, null); + assertFalse(httpGet.isAborted()); + httpGet.abort(); + assertTrue(httpGet.isAborted()); + try { + assertTrue(future.isCancelled()); + future.get(); + throw new AssertionError("exception should have been thrown"); + } catch(CancellationException e) { + //expected + } + } + { + httpGet.reset(); + assertFalse(httpGet.isAborted()); + Future future = client.execute(httpHost, httpGet, null); + assertFalse(httpGet.isAborted()); + assertEquals(200, future.get().getStatusLine().getStatusCode()); + assertFalse(future.isCancelled()); + } + } + } + /** * End to end test for headers. We test it explicitly against a real http client as there are different ways * to set/add headers to the {@link org.apache.http.client.HttpClient}. @@ -356,7 +458,6 @@ public void testAuthCredentialsAreNotClearedOnAuthChallenge() throws Exception { assertThat(response200.getHeader("Authorization"), startsWith("Basic")); } } - } public void testUrlWithoutLeadingSlash() throws Exception { diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java index 3894fca7d0a47..dd133f90daadb 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java @@ -52,7 +52,6 @@ import org.junit.After; import org.junit.Before; import org.mockito.ArgumentCaptor; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import javax.net.ssl.SSLHandshakeException; @@ -68,7 +67,6 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -126,30 +124,24 @@ public void createRestClient() { static CloseableHttpAsyncClient mockHttpClient(final ExecutorService exec) { CloseableHttpAsyncClient httpClient = mock(CloseableHttpAsyncClient.class); when(httpClient.execute(any(HttpAsyncRequestProducer.class), any(HttpAsyncResponseConsumer.class), - any(HttpClientContext.class), any(FutureCallback.class))).thenAnswer(new Answer>() { - @Override - public Future answer(InvocationOnMock invocationOnMock) throws Throwable { + any(HttpClientContext.class), any(FutureCallback.class))).thenAnswer((Answer>) invocationOnMock -> { final HttpAsyncRequestProducer requestProducer = (HttpAsyncRequestProducer) invocationOnMock.getArguments()[0]; final FutureCallback futureCallback = (FutureCallback) invocationOnMock.getArguments()[3]; // Call the callback asynchronous to better simulate how async http client works - return exec.submit(new Callable() { - @Override - public HttpResponse call() throws Exception { - if (futureCallback != null) { - try { - HttpResponse httpResponse = responseOrException(requestProducer); - futureCallback.completed(httpResponse); - } catch(Exception e) { - futureCallback.failed(e); - } - return null; + return exec.submit(() -> { + if (futureCallback != null) { + try { + HttpResponse httpResponse = responseOrException(requestProducer); + futureCallback.completed(httpResponse); + } catch(Exception e) { + futureCallback.failed(e); } - return responseOrException(requestProducer); + return null; } + return responseOrException(requestProducer); }); - } - }); + }); return httpClient; } diff --git a/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java b/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java index 8653db4226fe1..7ade990e1f811 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java +++ b/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java @@ -36,6 +36,7 @@ import org.apache.http.ssl.SSLContextBuilder; import org.apache.http.ssl.SSLContexts; import org.apache.http.util.EntityUtils; +import org.elasticsearch.client.Cancellable; import org.elasticsearch.client.HttpAsyncResponseConsumerFactory; import org.elasticsearch.client.Node; import org.elasticsearch.client.NodeSelector; @@ -206,16 +207,17 @@ public HttpAsyncClientBuilder customizeHttpClient( Request request = new Request( "GET", // <1> "/"); // <2> - restClient.performRequestAsync(request, new ResponseListener() { - @Override - public void onSuccess(Response response) { - // <3> - } + Cancellable cancellable = restClient.performRequestAsync(request, + new ResponseListener() { + @Override + public void onSuccess(Response response) { + // <3> + } - @Override - public void onFailure(Exception exception) { - // <4> - } + @Override + public void onFailure(Exception exception) { + // <4> + } }); //end::rest-client-async } @@ -271,6 +273,26 @@ public void onFailure(Exception exception) { latch.await(); //end::rest-client-async-example } + { + //tag::rest-client-async-cancel + Request request = new Request("GET", "/posts/_search"); + Cancellable cancellable = restClient.performRequestAsync( + request, + new ResponseListener() { + @Override + public void onSuccess(Response response) { + // <1> + } + + @Override + public void onFailure(Exception exception) { + // <2> + } + } + ); + cancellable.cancel(); + //end::rest-client-async-cancel + } { //tag::rest-client-response2 Response response = restClient.performRequest(new Request("GET", "/")); diff --git a/docs/java-rest/low-level/usage.asciidoc b/docs/java-rest/low-level/usage.asciidoc index 06bd77c7710dd..d0f4b070a55d6 100644 --- a/docs/java-rest/low-level/usage.asciidoc +++ b/docs/java-rest/low-level/usage.asciidoc @@ -224,7 +224,7 @@ Once the `RestClient` has been created, requests can be sent by calling either will block the calling thread and return the `Response` when the request is successful or throw an exception if it fails. `performRequestAsync` is asynchronous and accepts a `ResponseListener` argument that it calls with a -`Response` when the request is successful or with an `Exception` if it4 fails. +`Response` when the request is successful or with an `Exception` if it fails. This is synchronous: @@ -329,6 +329,22 @@ include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-async-examp <2> Handle the returned exception, due to communication error or a response with status code that indicates an error +==== Cancelling asynchronous requests + +The `performRequestAsync` method returns a `Cancellable` that exposes a single +public method called `cancel`. Such method can be called to cancel the on-going +request. Cancelling a request will result in aborting the http request through +the underlying http client. On the server side, this does not automatically +translate to the execution of that request being cancelled, which needs to be +specifically implemented in the API itself. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-async-cancel] +-------------------------------------------------- +<1> Process the returned response, in case it was ready before the request got cancelled +<2> Handle the returned exception, which will most likely be a `CancellationException` as the request got cancelled + [[java-rest-low-usage-responses]] === Reading responses From 73381a4e6ac54cd26afc6f3da4f27a044df7f449 Mon Sep 17 00:00:00 2001 From: Jilles van Gurp Date: Fri, 23 Aug 2019 16:38:15 +0200 Subject: [PATCH 2/5] Expose the ability to cancel async requests in REST high-level client (#45688) This commits makes all the async methods in the high level client return the `Cancellable` object that the low level client now exposes. Relates to #45379 Closes #44802 --- .../org/elasticsearch/client/CcrClient.java | 106 +++--- .../elasticsearch/client/ClusterClient.java | 22 +- .../elasticsearch/client/DataFrameClient.java | 66 ++-- .../org/elasticsearch/client/GraphClient.java | 15 +- .../client/IndexLifecycleClient.java | 94 +++-- .../elasticsearch/client/IndicesClient.java | 333 ++++++++++------- .../elasticsearch/client/IngestClient.java | 26 +- .../elasticsearch/client/LicenseClient.java | 33 +- .../client/MachineLearningClient.java | 334 +++++++++++------- .../elasticsearch/client/MigrationClient.java | 7 +- .../client/RestHighLevelClient.java | 308 +++++++++------- .../elasticsearch/client/RollupClient.java | 58 +-- .../elasticsearch/client/SecurityClient.java | 204 ++++++----- .../elasticsearch/client/SnapshotClient.java | 82 +++-- .../org/elasticsearch/client/TasksClient.java | 29 +- .../elasticsearch/client/WatcherClient.java | 63 ++-- .../org/elasticsearch/client/XPackClient.java | 12 +- .../client/RestHighLevelClientTests.java | 2 +- 18 files changed, 1044 insertions(+), 750 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrClient.java index 373b94124d43e..daa7c54b7fe3a 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/CcrClient.java @@ -85,15 +85,15 @@ public PutFollowResponse putFollow(PutFollowRequest request, RequestOptions opti * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putFollowAsync(PutFollowRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable putFollowAsync(PutFollowRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::putFollow, options, @@ -129,15 +129,15 @@ public AcknowledgedResponse pauseFollow(PauseFollowRequest request, RequestOptio * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void pauseFollowAsync(PauseFollowRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable pauseFollowAsync(PauseFollowRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::pauseFollow, options, @@ -172,15 +172,15 @@ public AcknowledgedResponse resumeFollow(ResumeFollowRequest request, RequestOpt * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void resumeFollowAsync(ResumeFollowRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable resumeFollowAsync(ResumeFollowRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::resumeFollow, options, @@ -217,15 +217,15 @@ public AcknowledgedResponse unfollow(UnfollowRequest request, RequestOptions opt * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void unfollowAsync(UnfollowRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable unfollowAsync(UnfollowRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::unfollow, options, @@ -260,15 +260,15 @@ public BroadcastResponse forgetFollower(final ForgetFollowerRequest request, fin * * See the docs for more details * on the intended usage of this API. - * * @param request the request * @param options the request options (e.g., headers), use {@link RequestOptions#DEFAULT} if the defaults are acceptable. + * @return cancellable that may be used to cancel the request */ - public void forgetFollowerAsync( + public Cancellable forgetFollowerAsync( final ForgetFollowerRequest request, final RequestOptions options, final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::forgetFollower, options, @@ -303,15 +303,15 @@ public AcknowledgedResponse putAutoFollowPattern(PutAutoFollowPatternRequest req * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putAutoFollowPatternAsync(PutAutoFollowPatternRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable putAutoFollowPatternAsync(PutAutoFollowPatternRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::putAutoFollowPattern, options, @@ -347,15 +347,15 @@ public AcknowledgedResponse deleteAutoFollowPattern(DeleteAutoFollowPatternReque * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteAutoFollowPatternAsync(DeleteAutoFollowPatternRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable deleteAutoFollowPatternAsync(DeleteAutoFollowPatternRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::deleteAutoFollowPattern, options, @@ -392,15 +392,15 @@ public GetAutoFollowPatternResponse getAutoFollowPattern(GetAutoFollowPatternReq * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getAutoFollowPatternAsync(GetAutoFollowPatternRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable getAutoFollowPatternAsync(GetAutoFollowPatternRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::getAutoFollowPattern, options, @@ -437,14 +437,14 @@ public CcrStatsResponse getCcrStats(CcrStatsRequest request, * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return cancellable that may be used to cancel the request */ - public void getCcrStatsAsync(CcrStatsRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable getCcrStatsAsync(CcrStatsRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::getCcrStats, options, @@ -481,14 +481,14 @@ public FollowStatsResponse getFollowStats(FollowStatsRequest request, * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return cancellable that may be used to cancel the request */ - public void getFollowStatsAsync(FollowStatsRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable getFollowStatsAsync(FollowStatsRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::getFollowStats, options, @@ -524,14 +524,14 @@ public FollowInfoResponse getFollowInfo(FollowInfoRequest request, RequestOption * * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return cancellable that may be used to cancel the request */ - public void getFollowInfoAsync(FollowInfoRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable getFollowInfoAsync(FollowInfoRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, CcrRequestConverters::getFollowInfo, options, diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java index f9b1474c69ae4..5e99975f51491 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java @@ -67,10 +67,12 @@ public ClusterUpdateSettingsResponse putSettings(ClusterUpdateSettingsRequest cl * @param clusterUpdateSettingsRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putSettingsAsync(ClusterUpdateSettingsRequest clusterUpdateSettingsRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(clusterUpdateSettingsRequest, ClusterRequestConverters::clusterPutSettings, + public Cancellable putSettingsAsync(ClusterUpdateSettingsRequest clusterUpdateSettingsRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(clusterUpdateSettingsRequest, + ClusterRequestConverters::clusterPutSettings, options, ClusterUpdateSettingsResponse::fromXContent, listener, emptySet()); } @@ -96,10 +98,12 @@ public ClusterGetSettingsResponse getSettings(ClusterGetSettingsRequest clusterG * @param clusterGetSettingsRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getSettingsAsync(ClusterGetSettingsRequest clusterGetSettingsRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(clusterGetSettingsRequest, ClusterRequestConverters::clusterGetSettings, + public Cancellable getSettingsAsync(ClusterGetSettingsRequest clusterGetSettingsRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + clusterGetSettingsRequest, ClusterRequestConverters::clusterGetSettings, options, ClusterGetSettingsResponse::fromXContent, listener, emptySet()); } @@ -127,9 +131,11 @@ public ClusterHealthResponse health(ClusterHealthRequest healthRequest, RequestO * @param healthRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void healthAsync(ClusterHealthRequest healthRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(healthRequest, ClusterRequestConverters::clusterHealth, options, + public Cancellable healthAsync(ClusterHealthRequest healthRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(healthRequest, ClusterRequestConverters::clusterHealth, options, ClusterHealthResponse::fromXContent, listener, singleton(RestStatus.REQUEST_TIMEOUT.getStatus())); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java index f5104c1602205..2570bdee0aa5b 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/DataFrameClient.java @@ -74,15 +74,15 @@ public AcknowledgedResponse putDataFrameTransform(PutDataFrameTransformRequest r * For additional info * see * Create data frame transform documentation - * * @param request The PutDataFrameTransformRequest containing the * {@link org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfig}. * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putDataFrameTransformAsync(PutDataFrameTransformRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putDataFrameTransformAsync(PutDataFrameTransformRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::putDataFrameTransform, options, AcknowledgedResponse::fromXContent, @@ -118,16 +118,16 @@ public UpdateDataFrameTransformResponse updateDataFrameTransform(UpdateDataFrame * For additional info * see * Create data frame transform documentation - * * @param request The UpdateDataFrameTransformRequest containing the * {@link org.elasticsearch.client.dataframe.transforms.DataFrameTransformConfigUpdate}. * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void updateDataFrameTransformAsync(UpdateDataFrameTransformRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable updateDataFrameTransformAsync(UpdateDataFrameTransformRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::updateDataFrameTransform, options, UpdateDataFrameTransformResponse::fromXContent, @@ -162,14 +162,14 @@ public GetDataFrameTransformStatsResponse getDataFrameTransformStats(GetDataFram * For additional info * see * Get data frame transform stats documentation - * * @param request Specifies the which transforms to get the stats for * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getDataFrameTransformStatsAsync(GetDataFrameTransformStatsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getDataFrameTransformStatsAsync(GetDataFrameTransformStatsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::getDataFrameTransformStats, options, GetDataFrameTransformStatsResponse::fromXContent, @@ -204,14 +204,14 @@ public AcknowledgedResponse deleteDataFrameTransform(DeleteDataFrameTransformReq * For additional info * see * Delete data frame transform documentation - * * @param request The delete data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteDataFrameTransformAsync(DeleteDataFrameTransformRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteDataFrameTransformAsync(DeleteDataFrameTransformRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::deleteDataFrameTransform, options, AcknowledgedResponse::fromXContent, @@ -245,14 +245,14 @@ public PreviewDataFrameTransformResponse previewDataFrameTransform(PreviewDataFr *

* see * Preview data frame transform documentation - * * @param request The preview data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void previewDataFrameTransformAsync(PreviewDataFrameTransformRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable previewDataFrameTransformAsync(PreviewDataFrameTransformRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::previewDataFrameTransform, options, PreviewDataFrameTransformResponse::fromXContent, @@ -287,14 +287,14 @@ public StartDataFrameTransformResponse startDataFrameTransform(StartDataFrameTra * For additional info * see * Start data frame transform documentation - * * @param request The start data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void startDataFrameTransformAsync(StartDataFrameTransformRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable startDataFrameTransformAsync(StartDataFrameTransformRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::startDataFrameTransform, options, StartDataFrameTransformResponse::fromXContent, @@ -329,14 +329,14 @@ public StopDataFrameTransformResponse stopDataFrameTransform(StopDataFrameTransf * For additional info * see * Stop data frame transform documentation - * * @param request The stop data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void stopDataFrameTransformAsync(StopDataFrameTransformRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable stopDataFrameTransformAsync(StopDataFrameTransformRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::stopDataFrameTransform, options, StopDataFrameTransformResponse::fromXContent, @@ -371,14 +371,14 @@ public GetDataFrameTransformResponse getDataFrameTransform(GetDataFrameTransform * For additional info * see * Get data frame transform documentation - * * @param request The get data frame transform request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getDataFrameTransformAsync(GetDataFrameTransformRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getDataFrameTransformAsync(GetDataFrameTransformRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, DataFrameRequestConverters::getDataFrameTransform, options, GetDataFrameTransformResponse::fromXContent, diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/GraphClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/GraphClient.java index 70912b094d023..44b3b8cf4d04d 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/GraphClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/GraphClient.java @@ -34,7 +34,7 @@ public class GraphClient { GraphClient(RestHighLevelClient restHighLevelClient) { this.restHighLevelClient = restHighLevelClient; } - + /** * Executes an exploration request using the Graph API. * @@ -52,12 +52,13 @@ public final GraphExploreResponse explore(GraphExploreRequest graphExploreReques * * See Graph API * on elastic.co. + * @return cancellable that may be used to cancel the request */ - public final void exploreAsync(GraphExploreRequest graphExploreRequest, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(graphExploreRequest, GraphRequestConverters::explore, + public final Cancellable exploreAsync(GraphExploreRequest graphExploreRequest, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(graphExploreRequest, GraphRequestConverters::explore, options, GraphExploreResponse::fromXContent, listener, emptySet()); - } - + } + } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java index a3fd10bee6e6e..c7e3a8e0edabb 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java @@ -76,10 +76,11 @@ public GetLifecyclePolicyResponse getLifecyclePolicy(GetLifecyclePolicyRequest r * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getLifecyclePolicyAsync(GetLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::getLifecyclePolicy, options, + public Cancellable getLifecyclePolicyAsync(GetLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::getLifecyclePolicy, options, GetLifecyclePolicyResponse::fromXContent, listener, emptySet()); } @@ -105,10 +106,11 @@ public AcknowledgedResponse putLifecyclePolicy(PutLifecyclePolicyRequest request * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putLifecyclePolicyAsync(PutLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::putLifecyclePolicy, options, + public Cancellable putLifecyclePolicyAsync(PutLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::putLifecyclePolicy, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -134,10 +136,12 @@ public AcknowledgedResponse deleteLifecyclePolicy(DeleteLifecyclePolicyRequest r * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteLifecyclePolicyAsync(DeleteLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::deleteLifecyclePolicy, options, + public Cancellable deleteLifecyclePolicyAsync(DeleteLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + request, IndexLifecycleRequestConverters::deleteLifecyclePolicy, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -163,10 +167,12 @@ public RemoveIndexLifecyclePolicyResponse removeIndexLifecyclePolicy(RemoveIndex * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void removeIndexLifecyclePolicyAsync(RemoveIndexLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::removeIndexLifecyclePolicy, options, + public Cancellable removeIndexLifecyclePolicyAsync(RemoveIndexLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + request, IndexLifecycleRequestConverters::removeIndexLifecyclePolicy, options, RemoveIndexLifecyclePolicyResponse::fromXContent, listener, emptySet()); } @@ -191,9 +197,10 @@ public AcknowledgedResponse startILM(StartILMRequest request, RequestOptions opt * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void startILMAsync(StartILMRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::startILM, options, + public Cancellable startILMAsync(StartILMRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::startILM, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -229,14 +236,15 @@ public LifecycleManagementStatusResponse lifecycleManagementStatus(LifecycleMana * Asynchronously get the status of index lifecycle management * See * the docs for more. - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void lifecycleManagementStatusAsync(LifecycleManagementStatusRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::lifecycleManagementStatus, options, + public Cancellable lifecycleManagementStatusAsync(LifecycleManagementStatusRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + request, IndexLifecycleRequestConverters::lifecycleManagementStatus, options, LifecycleManagementStatusResponse::fromXContent, listener, emptySet()); } @@ -247,9 +255,10 @@ public void lifecycleManagementStatusAsync(LifecycleManagementStatusRequest requ * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void stopILMAsync(StopILMRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::stopILM, options, + public Cancellable stopILMAsync(StopILMRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::stopILM, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -274,10 +283,11 @@ public ExplainLifecycleResponse explainLifecycle(ExplainLifecycleRequest request * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void explainLifecycleAsync(ExplainLifecycleRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::explainLifecycle, options, + public Cancellable explainLifecycleAsync(ExplainLifecycleRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::explainLifecycle, options, ExplainLifecycleResponse::fromXContent, listener, emptySet()); } @@ -302,10 +312,11 @@ public AcknowledgedResponse retryLifecyclePolicy(RetryLifecyclePolicyRequest req * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void retryLifecyclePolicyAsync(RetryLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::retryLifecycle, options, + public Cancellable retryLifecyclePolicyAsync(RetryLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::retryLifecycle, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -337,10 +348,11 @@ public GetSnapshotLifecyclePolicyResponse getSnapshotLifecyclePolicy(GetSnapshot * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getSnapshotLifecyclePolicyAsync(GetSnapshotLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::getSnapshotLifecyclePolicy, + public Cancellable getSnapshotLifecyclePolicyAsync(GetSnapshotLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::getSnapshotLifecyclePolicy, options, GetSnapshotLifecyclePolicyResponse::fromXContent, listener, emptySet()); } @@ -372,10 +384,11 @@ public AcknowledgedResponse putSnapshotLifecyclePolicy(PutSnapshotLifecyclePolic * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putSnapshotLifecyclePolicyAsync(PutSnapshotLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::putSnapshotLifecyclePolicy, + public Cancellable putSnapshotLifecyclePolicyAsync(PutSnapshotLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::putSnapshotLifecyclePolicy, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -407,10 +420,12 @@ public AcknowledgedResponse deleteSnapshotLifecyclePolicy(DeleteSnapshotLifecycl * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteSnapshotLifecyclePolicyAsync(DeleteSnapshotLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::deleteSnapshotLifecyclePolicy, + public Cancellable deleteSnapshotLifecyclePolicyAsync(DeleteSnapshotLifecyclePolicyRequest request, + RequestOptions options,ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + request, IndexLifecycleRequestConverters::deleteSnapshotLifecyclePolicy, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -442,10 +457,13 @@ public ExecuteSnapshotLifecyclePolicyResponse executeSnapshotLifecyclePolicy(Exe * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void executeSnapshotLifecyclePolicyAsync(ExecuteSnapshotLifecyclePolicyRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::executeSnapshotLifecyclePolicy, + public Cancellable executeSnapshotLifecyclePolicyAsync( + ExecuteSnapshotLifecyclePolicyRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + request, IndexLifecycleRequestConverters::executeSnapshotLifecyclePolicy, options, ExecuteSnapshotLifecyclePolicyResponse::fromXContent, listener, emptySet()); } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java index 9394495313df4..c83bcbc9c3da1 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesClient.java @@ -108,9 +108,12 @@ public AcknowledgedResponse delete(DeleteIndexRequest deleteIndexRequest, Reques * @param deleteIndexRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteAsync(DeleteIndexRequest deleteIndexRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(deleteIndexRequest, IndicesRequestConverters::deleteIndex, options, + public Cancellable deleteAsync(DeleteIndexRequest deleteIndexRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(deleteIndexRequest, + IndicesRequestConverters::deleteIndex, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -136,11 +139,12 @@ public CreateIndexResponse create(CreateIndexRequest createIndexRequest, * @param createIndexRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void createAsync(CreateIndexRequest createIndexRequest, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(createIndexRequest, IndicesRequestConverters::createIndex, options, + public Cancellable createAsync(CreateIndexRequest createIndexRequest, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(createIndexRequest, IndicesRequestConverters::createIndex, options, CreateIndexResponse::fromXContent, listener, emptySet()); } @@ -177,12 +181,13 @@ public org.elasticsearch.action.admin.indices.create.CreateIndexResponse create( * @deprecated This method uses an old request object which still refers to types, a deprecated feature. The * method {@link #createAsync(CreateIndexRequest, RequestOptions, ActionListener)} should be used instead, * which accepts a new request object. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void createAsync(org.elasticsearch.action.admin.indices.create.CreateIndexRequest createIndexRequest, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(createIndexRequest, + public Cancellable createAsync(org.elasticsearch.action.admin.indices.create.CreateIndexRequest createIndexRequest, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(createIndexRequest, IndicesRequestConverters::createIndex, options, org.elasticsearch.action.admin.indices.create.CreateIndexResponse::fromXContent, listener, emptySet()); } @@ -208,10 +213,11 @@ public AcknowledgedResponse putMapping(PutMappingRequest putMappingRequest, Requ * @param putMappingRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putMappingAsync(PutMappingRequest putMappingRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(putMappingRequest, IndicesRequestConverters::putMapping, options, + public Cancellable putMappingAsync(PutMappingRequest putMappingRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(putMappingRequest, IndicesRequestConverters::putMapping, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -245,12 +251,13 @@ public AcknowledgedResponse putMapping(org.elasticsearch.action.admin.indices.ma * @deprecated This method uses an old request object which still refers to types, a deprecated feature. The * method {@link #putMappingAsync(PutMappingRequest, RequestOptions, ActionListener)} should be used instead, * which accepts a new request object. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void putMappingAsync(org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest putMappingRequest, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(putMappingRequest, IndicesRequestConverters::putMapping, options, + public Cancellable putMappingAsync(org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest putMappingRequest, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(putMappingRequest, IndicesRequestConverters::putMapping, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -278,10 +285,11 @@ public GetMappingsResponse getMapping(GetMappingsRequest getMappingsRequest, Req * @param getMappingsRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getMappingAsync(GetMappingsRequest getMappingsRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getMappingsRequest, + public Cancellable getMappingAsync(GetMappingsRequest getMappingsRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getMappingsRequest, IndicesRequestConverters::getMappings, options, GetMappingsResponse::fromXContent, @@ -324,12 +332,13 @@ public org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse ge * @deprecated This method uses old request and response objects which still refer to types, a deprecated feature. * The method {@link #getMapping(GetMappingsRequest, RequestOptions)} should be used instead, which accepts a new * request object. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void getMappingAsync(org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest getMappingsRequest, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getMappingsRequest, + public Cancellable getMappingAsync(org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest getMappingsRequest, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getMappingsRequest, IndicesRequestConverters::getMappings, options, org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse::fromXContent, @@ -369,13 +378,17 @@ public org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsRespon * @deprecated This method uses old request and response objects which still refer to types, a deprecated feature. * The method {@link #getFieldMappingAsync(GetFieldMappingsRequest, RequestOptions, ActionListener)} should be * used instead, which accepts a new request object. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void getFieldMappingAsync(org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsRequest getFieldMappingsRequest, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getFieldMappingsRequest, IndicesRequestConverters::getFieldMapping, options, - org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse::fromXContent, listener, emptySet()); + public Cancellable getFieldMappingAsync( + org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsRequest getFieldMappingsRequest, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getFieldMappingsRequest, + IndicesRequestConverters::getFieldMapping, options, + org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse::fromXContent, + listener, emptySet()); } /** @@ -401,10 +414,12 @@ public GetFieldMappingsResponse getFieldMapping(GetFieldMappingsRequest getField * @param getFieldMappingsRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getFieldMappingAsync(GetFieldMappingsRequest getFieldMappingsRequest, - RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getFieldMappingsRequest, IndicesRequestConverters::getFieldMapping, options, + public Cancellable getFieldMappingAsync(GetFieldMappingsRequest getFieldMappingsRequest, + RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + getFieldMappingsRequest, IndicesRequestConverters::getFieldMapping, options, GetFieldMappingsResponse::fromXContent, listener, emptySet()); } @@ -429,10 +444,12 @@ public AcknowledgedResponse updateAliases(IndicesAliasesRequest indicesAliasesRe * @param indicesAliasesRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void updateAliasesAsync(IndicesAliasesRequest indicesAliasesRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(indicesAliasesRequest, IndicesRequestConverters::updateAliases, options, + public Cancellable updateAliasesAsync(IndicesAliasesRequest indicesAliasesRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(indicesAliasesRequest, + IndicesRequestConverters::updateAliases, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -457,9 +474,10 @@ public OpenIndexResponse open(OpenIndexRequest openIndexRequest, RequestOptions * @param openIndexRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void openAsync(OpenIndexRequest openIndexRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(openIndexRequest, IndicesRequestConverters::openIndex, options, + public Cancellable openAsync(OpenIndexRequest openIndexRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(openIndexRequest, IndicesRequestConverters::openIndex, options, OpenIndexResponse::fromXContent, listener, emptySet()); } @@ -484,9 +502,12 @@ public CloseIndexResponse close(CloseIndexRequest closeIndexRequest, RequestOpti * @param closeIndexRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void closeAsync(CloseIndexRequest closeIndexRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(closeIndexRequest, IndicesRequestConverters::closeIndex, options, + public Cancellable closeAsync(CloseIndexRequest closeIndexRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(closeIndexRequest, + IndicesRequestConverters::closeIndex, options, CloseIndexResponse::fromXContent, listener, emptySet()); } @@ -512,9 +533,10 @@ public boolean existsAlias(GetAliasesRequest getAliasesRequest, RequestOptions o * @param getAliasesRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void existsAliasAsync(GetAliasesRequest getAliasesRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsync(getAliasesRequest, IndicesRequestConverters::existsAlias, options, + public Cancellable existsAliasAsync(GetAliasesRequest getAliasesRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsync(getAliasesRequest, IndicesRequestConverters::existsAlias, options, RestHighLevelClient::convertExistsResponse, listener, emptySet()); } @@ -537,9 +559,10 @@ public RefreshResponse refresh(RefreshRequest refreshRequest, RequestOptions opt * @param refreshRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void refreshAsync(RefreshRequest refreshRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(refreshRequest, IndicesRequestConverters::refresh, options, + public Cancellable refreshAsync(RefreshRequest refreshRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(refreshRequest, IndicesRequestConverters::refresh, options, RefreshResponse::fromXContent, listener, emptySet()); } @@ -562,9 +585,10 @@ public FlushResponse flush(FlushRequest flushRequest, RequestOptions options) th * @param flushRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void flushAsync(FlushRequest flushRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(flushRequest, IndicesRequestConverters::flush, options, + public Cancellable flushAsync(FlushRequest flushRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(flushRequest, IndicesRequestConverters::flush, options, FlushResponse::fromXContent, listener, emptySet()); } @@ -589,10 +613,11 @@ public SyncedFlushResponse flushSynced(SyncedFlushRequest syncedFlushRequest, Re * @param syncedFlushRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void flushSyncedAsync(SyncedFlushRequest syncedFlushRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(syncedFlushRequest, IndicesRequestConverters::flushSynced, options, + public Cancellable flushSyncedAsync(SyncedFlushRequest syncedFlushRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(syncedFlushRequest, IndicesRequestConverters::flushSynced, options, SyncedFlushResponse::fromXContent, listener, emptySet()); } @@ -617,10 +642,11 @@ public GetSettingsResponse getSettings(GetSettingsRequest getSettingsRequest, Re * @param getSettingsRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getSettingsAsync(GetSettingsRequest getSettingsRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getSettingsRequest, IndicesRequestConverters::getSettings, options, + public Cancellable getSettingsAsync(GetSettingsRequest getSettingsRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getSettingsRequest, IndicesRequestConverters::getSettings, options, GetSettingsResponse::fromXContent, listener, emptySet()); } @@ -645,10 +671,11 @@ public GetIndexResponse get(GetIndexRequest getIndexRequest, RequestOptions opti * @param getIndexRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getAsync(GetIndexRequest getIndexRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getIndexRequest, IndicesRequestConverters::getIndex, options, + public Cancellable getAsync(GetIndexRequest getIndexRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getIndexRequest, IndicesRequestConverters::getIndex, options, GetIndexResponse::fromXContent, listener, emptySet()); } @@ -679,11 +706,12 @@ public org.elasticsearch.action.admin.indices.get.GetIndexResponse get( * @param listener the listener to be notified upon request completion * @deprecated This method uses an old request object which still refers to types, a deprecated feature. The method * {@link #getAsync(GetIndexRequest, RequestOptions, ActionListener)} should be used instead, which accepts a new request object. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void getAsync(org.elasticsearch.action.admin.indices.get.GetIndexRequest getIndexRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getIndexRequest, IndicesRequestConverters::getIndex, options, + public Cancellable getAsync(org.elasticsearch.action.admin.indices.get.GetIndexRequest getIndexRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getIndexRequest, IndicesRequestConverters::getIndex, options, org.elasticsearch.action.admin.indices.get.GetIndexResponse::fromXContent, listener, emptySet()); } @@ -724,10 +752,12 @@ public ForceMergeResponse forcemerge(ForceMergeRequest forceMergeRequest, Reques * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion * @deprecated use {@link #forcemergeAsync(ForceMergeRequest, RequestOptions, ActionListener)} instead + * @return cancellable that may be used to cancel the request */ @Deprecated - public void forceMergeAsync(ForceMergeRequest forceMergeRequest, RequestOptions options, ActionListener listener) { - forcemergeAsync(forceMergeRequest, options, listener); + public Cancellable forceMergeAsync(ForceMergeRequest forceMergeRequest, RequestOptions options, + ActionListener listener) { + return forcemergeAsync(forceMergeRequest, options, listener); } /** @@ -737,9 +767,12 @@ public void forceMergeAsync(ForceMergeRequest forceMergeRequest, RequestOptions * @param forceMergeRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void forcemergeAsync(ForceMergeRequest forceMergeRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(forceMergeRequest, IndicesRequestConverters::forceMerge, options, + public Cancellable forcemergeAsync(ForceMergeRequest forceMergeRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(forceMergeRequest, + IndicesRequestConverters::forceMerge, options, ForceMergeResponse::fromXContent, listener, emptySet()); } @@ -765,10 +798,12 @@ public ClearIndicesCacheResponse clearCache(ClearIndicesCacheRequest clearIndice * @param clearIndicesCacheRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void clearCacheAsync(ClearIndicesCacheRequest clearIndicesCacheRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(clearIndicesCacheRequest, IndicesRequestConverters::clearCache, options, + public Cancellable clearCacheAsync(ClearIndicesCacheRequest clearIndicesCacheRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(clearIndicesCacheRequest, + IndicesRequestConverters::clearCache, options, ClearIndicesCacheResponse::fromXContent, listener, emptySet()); } @@ -798,9 +833,10 @@ public boolean exists(GetIndexRequest request, RequestOptions options) throws IO * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void existsAsync(GetIndexRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsync( + public Cancellable existsAsync(GetIndexRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsync( request, IndicesRequestConverters::indicesExist, options, @@ -841,11 +877,12 @@ public boolean exists(org.elasticsearch.action.admin.indices.get.GetIndexRequest * @param listener the listener to be notified upon request completion * @deprecated This method uses an old request object which still refers to types, a deprecated feature. The method * {@link #existsAsync(GetIndexRequest, RequestOptions, ActionListener)} should be used instead, which accepts a new request object. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void existsAsync(org.elasticsearch.action.admin.indices.get.GetIndexRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsync( + public Cancellable existsAsync(org.elasticsearch.action.admin.indices.get.GetIndexRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsync( request, IndicesRequestConverters::indicesExist, options, @@ -876,9 +913,10 @@ public ResizeResponse shrink(ResizeRequest resizeRequest, RequestOptions options * @param resizeRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void shrinkAsync(ResizeRequest resizeRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(resizeRequest, IndicesRequestConverters::shrink, options, + public Cancellable shrinkAsync(ResizeRequest resizeRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(resizeRequest, IndicesRequestConverters::shrink, options, ResizeResponse::fromXContent, listener, emptySet()); } @@ -903,9 +941,10 @@ public ResizeResponse split(ResizeRequest resizeRequest, RequestOptions options) * @param resizeRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void splitAsync(ResizeRequest resizeRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(resizeRequest, IndicesRequestConverters::split, options, + public Cancellable splitAsync(ResizeRequest resizeRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(resizeRequest, IndicesRequestConverters::split, options, ResizeResponse::fromXContent, listener, emptySet()); } @@ -930,9 +969,10 @@ public ResizeResponse clone(ResizeRequest resizeRequest, RequestOptions options) * @param resizeRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void cloneAsync(ResizeRequest resizeRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(resizeRequest, IndicesRequestConverters::clone, options, + public Cancellable cloneAsync(ResizeRequest resizeRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(resizeRequest, IndicesRequestConverters::clone, options, ResizeResponse::fromXContent, listener, emptySet()); } @@ -957,9 +997,10 @@ public RolloverResponse rollover(RolloverRequest rolloverRequest, RequestOptions * @param rolloverRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void rolloverAsync(RolloverRequest rolloverRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(rolloverRequest, IndicesRequestConverters::rollover, options, + public Cancellable rolloverAsync(RolloverRequest rolloverRequest, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(rolloverRequest, IndicesRequestConverters::rollover, options, RolloverResponse::fromXContent, listener, emptySet()); } @@ -995,11 +1036,13 @@ public org.elasticsearch.action.admin.indices.rollover.RolloverResponse rollover * @deprecated This method uses deprecated request and response objects. * The method {@link #rolloverAsync(RolloverRequest, RequestOptions, ActionListener)} should be used instead, which * accepts a new request object. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void rolloverAsync(org.elasticsearch.action.admin.indices.rollover.RolloverRequest rolloverRequest, - RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(rolloverRequest, IndicesRequestConverters::rollover, options, + public Cancellable rolloverAsync(org.elasticsearch.action.admin.indices.rollover.RolloverRequest rolloverRequest, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(rolloverRequest, IndicesRequestConverters::rollover, options, org.elasticsearch.action.admin.indices.rollover.RolloverResponse::fromXContent, listener, emptySet()); } @@ -1024,10 +1067,13 @@ public GetAliasesResponse getAlias(GetAliasesRequest getAliasesRequest, RequestO * @param getAliasesRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getAliasAsync(GetAliasesRequest getAliasesRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getAliasesRequest, IndicesRequestConverters::getAlias, options, - GetAliasesResponse::fromXContent, listener, singleton(RestStatus.NOT_FOUND.getStatus())); + public Cancellable getAliasAsync(GetAliasesRequest getAliasesRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getAliasesRequest, + IndicesRequestConverters::getAlias, options, + GetAliasesResponse::fromXContent, listener, singleton(RestStatus.NOT_FOUND.getStatus())); } /** @@ -1051,10 +1097,12 @@ public AcknowledgedResponse putSettings(UpdateSettingsRequest updateSettingsRequ * @param updateSettingsRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putSettingsAsync(UpdateSettingsRequest updateSettingsRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(updateSettingsRequest, IndicesRequestConverters::indexPutSettings, options, + public Cancellable putSettingsAsync(UpdateSettingsRequest updateSettingsRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(updateSettingsRequest, + IndicesRequestConverters::indexPutSettings, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1088,11 +1136,14 @@ public AcknowledgedResponse putTemplate( * @deprecated This old form of request allows types in mappings. * Use {@link #putTemplateAsync(PutIndexTemplateRequest, RequestOptions, ActionListener)} * instead which introduces a new request object without types. + * @return cancellable that may be used to cancel the request */ @Deprecated - public void putTemplateAsync(org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest putIndexTemplateRequest, - RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(putIndexTemplateRequest, IndicesRequestConverters::putTemplate, options, + public Cancellable putTemplateAsync( + org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest putIndexTemplateRequest, + RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(putIndexTemplateRequest, + IndicesRequestConverters::putTemplate, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1120,10 +1171,12 @@ public AcknowledgedResponse putTemplate( * @param putIndexTemplateRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putTemplateAsync(PutIndexTemplateRequest putIndexTemplateRequest, - RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(putIndexTemplateRequest, IndicesRequestConverters::putTemplate, options, + public Cancellable putTemplateAsync(PutIndexTemplateRequest putIndexTemplateRequest, + RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(putIndexTemplateRequest, + IndicesRequestConverters::putTemplate, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1137,8 +1190,10 @@ public void putTemplateAsync(PutIndexTemplateRequest putIndexTemplateRequest, * @return the response * @throws IOException in case there is a problem sending the request or parsing back the response */ - public ValidateQueryResponse validateQuery(ValidateQueryRequest validateQueryRequest, RequestOptions options) throws IOException { - return restHighLevelClient.performRequestAndParseEntity(validateQueryRequest, IndicesRequestConverters::validateQuery, options, + public ValidateQueryResponse validateQuery(ValidateQueryRequest validateQueryRequest, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequestAndParseEntity(validateQueryRequest, + IndicesRequestConverters::validateQuery, options, ValidateQueryResponse::fromXContent, emptySet()); } @@ -1150,10 +1205,12 @@ public ValidateQueryResponse validateQuery(ValidateQueryRequest validateQueryReq * @param validateQueryRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void validateQueryAsync(ValidateQueryRequest validateQueryRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(validateQueryRequest, IndicesRequestConverters::validateQuery, options, + public Cancellable validateQueryAsync(ValidateQueryRequest validateQueryRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(validateQueryRequest, + IndicesRequestConverters::validateQuery, options, ValidateQueryResponse::fromXContent, listener, emptySet()); } @@ -1174,7 +1231,8 @@ public org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResp GetIndexTemplatesRequest getIndexTemplatesRequest, RequestOptions options) throws IOException { return restHighLevelClient.performRequestAndParseEntity(getIndexTemplatesRequest, IndicesRequestConverters::getTemplatesWithDocumentTypes, - options, org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse::fromXContent, emptySet()); + options, + org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse::fromXContent, emptySet()); } /** @@ -1186,8 +1244,8 @@ public org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResp * @return the response * @throws IOException in case there is a problem sending the request or parsing back the response */ - public GetIndexTemplatesResponse getIndexTemplate(GetIndexTemplatesRequest getIndexTemplatesRequest, RequestOptions options) - throws IOException { + public GetIndexTemplatesResponse getIndexTemplate(GetIndexTemplatesRequest getIndexTemplatesRequest, + RequestOptions options) throws IOException { return restHighLevelClient.performRequestAndParseEntity(getIndexTemplatesRequest, IndicesRequestConverters::getTemplates, options, GetIndexTemplatesResponse::fromXContent, emptySet()); @@ -1203,13 +1261,16 @@ public GetIndexTemplatesResponse getIndexTemplate(GetIndexTemplatesRequest getIn * @param listener the listener to be notified upon request completion * @deprecated This method uses an old response object which still refers to types, a deprecated feature. Use * {@link #getIndexTemplateAsync(GetIndexTemplatesRequest, RequestOptions, ActionListener)} instead which returns a new response object + * @return cancellable that may be used to cancel the request */ @Deprecated - public void getTemplateAsync(GetIndexTemplatesRequest getIndexTemplatesRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getIndexTemplatesRequest, + public Cancellable getTemplateAsync(GetIndexTemplatesRequest getIndexTemplatesRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getIndexTemplatesRequest, IndicesRequestConverters::getTemplatesWithDocumentTypes, - options, org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse::fromXContent, listener, emptySet()); + options, + org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse::fromXContent, + listener, emptySet()); } /** @@ -1219,10 +1280,11 @@ public void getTemplateAsync(GetIndexTemplatesRequest getIndexTemplatesRequest, * @param getIndexTemplatesRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getIndexTemplateAsync(GetIndexTemplatesRequest getIndexTemplatesRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getIndexTemplatesRequest, + public Cancellable getIndexTemplateAsync(GetIndexTemplatesRequest getIndexTemplatesRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getIndexTemplatesRequest, IndicesRequestConverters::getTemplates, options, GetIndexTemplatesResponse::fromXContent, listener, emptySet()); } @@ -1235,24 +1297,26 @@ public void getIndexTemplateAsync(GetIndexTemplatesRequest getIndexTemplatesRequ * @return true if any index templates in the request exist, false otherwise * @throws IOException in case there is a problem sending the request or parsing back the response */ - public boolean existsTemplate(IndexTemplatesExistRequest indexTemplatesRequest, RequestOptions options) throws IOException { - return restHighLevelClient.performRequest(indexTemplatesRequest, IndicesRequestConverters::templatesExist, options, + public boolean existsTemplate(IndexTemplatesExistRequest indexTemplatesRequest, + RequestOptions options) throws IOException { + return restHighLevelClient.performRequest(indexTemplatesRequest, + IndicesRequestConverters::templatesExist, options, RestHighLevelClient::convertExistsResponse, emptySet()); } /** * Uses the Index Templates API to determine if index templates exist - * * @param indexTemplatesExistRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion. The listener will be called with the value {@code true} - * if any index templates in the request exist, false otherwise + * @return cancellable that may be used to cancel the request */ - public void existsTemplateAsync(IndexTemplatesExistRequest indexTemplatesExistRequest, - RequestOptions options, - ActionListener listener) { + public Cancellable existsTemplateAsync(IndexTemplatesExistRequest indexTemplatesExistRequest, + RequestOptions options, + ActionListener listener) { - restHighLevelClient.performRequestAsync(indexTemplatesExistRequest, IndicesRequestConverters::templatesExist, options, + return restHighLevelClient.performRequestAsync(indexTemplatesExistRequest, + IndicesRequestConverters::templatesExist, options, RestHighLevelClient::convertExistsResponse, listener, emptySet()); } @@ -1273,14 +1337,14 @@ public AnalyzeResponse analyze(AnalyzeRequest request, RequestOptions options) t * Asynchronously calls the analyze API * * See Analyze API on elastic.co - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void analyzeAsync(AnalyzeRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::analyze, options, + public Cancellable analyzeAsync(AnalyzeRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::analyze, options, AnalyzeResponse::fromXContent, listener, emptySet()); } @@ -1297,13 +1361,14 @@ public ShardsAcknowledgedResponse freeze(FreezeIndexRequest request, RequestOpti /** * Asynchronously calls the _freeze API - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void freezeAsync(FreezeIndexRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::freezeIndex, options, + public Cancellable freezeAsync(FreezeIndexRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::freezeIndex, options, ShardsAcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1320,13 +1385,15 @@ public ShardsAcknowledgedResponse unfreeze(UnfreezeIndexRequest request, Request /** * Asynchronously calls the _unfreeze API - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void unfreezeAsync(UnfreezeIndexRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::unfreezeIndex, options, + public Cancellable unfreezeAsync(UnfreezeIndexRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, + IndicesRequestConverters::unfreezeIndex, options, ShardsAcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1348,14 +1415,14 @@ public AcknowledgedResponse deleteTemplate(DeleteIndexTemplateRequest request, R * Asynchronously delete an index template using the Index Templates API * See Index Templates API * on elastic.co - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteTemplateAsync(DeleteIndexTemplateRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::deleteTemplate, + public Cancellable deleteTemplateAsync(DeleteIndexTemplateRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::deleteTemplate, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1372,14 +1439,14 @@ public ReloadAnalyzersResponse reloadAnalyzers(ReloadAnalyzersRequest request, R /** * Asynchronously calls the _reload_search_analyzers API - * * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void reloadAnalyzersAsync(ReloadAnalyzersRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::reloadAnalyzers, options, + public Cancellable reloadAnalyzersAsync(ReloadAnalyzersRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndicesRequestConverters::reloadAnalyzers, options, ReloadAnalyzersResponse::fromXContent, listener, emptySet()); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestClient.java index 035d1fd26fb78..cfe481bd69b9f 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IngestClient.java @@ -67,9 +67,10 @@ public AcknowledgedResponse putPipeline(PutPipelineRequest request, RequestOptio * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putPipelineAsync(PutPipelineRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( request, IngestRequestConverters::putPipeline, options, + public Cancellable putPipelineAsync(PutPipelineRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, IngestRequestConverters::putPipeline, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -94,9 +95,10 @@ public GetPipelineResponse getPipeline(GetPipelineRequest request, RequestOption * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getPipelineAsync(GetPipelineRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( request, IngestRequestConverters::getPipeline, options, + public Cancellable getPipelineAsync(GetPipelineRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, IngestRequestConverters::getPipeline, options, GetPipelineResponse::fromXContent, listener, Collections.singleton(404)); } @@ -123,9 +125,12 @@ public AcknowledgedResponse deletePipeline(DeletePipelineRequest request, Reques * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deletePipelineAsync(DeletePipelineRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( request, IngestRequestConverters::deletePipeline, options, + public Cancellable deletePipelineAsync(DeletePipelineRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, + IngestRequestConverters::deletePipeline, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -154,11 +159,12 @@ public SimulatePipelineResponse simulate(SimulatePipelineRequest request, Reques * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void simulateAsync(SimulatePipelineRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( request, IngestRequestConverters::simulatePipeline, options, + public Cancellable simulateAsync(SimulatePipelineRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, IngestRequestConverters::simulatePipeline, options, SimulatePipelineResponse::fromXContent, listener, emptySet()); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseClient.java index 0f74a003315de..e0db84e6b2bbd 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/LicenseClient.java @@ -80,9 +80,10 @@ public PutLicenseResponse putLicense(PutLicenseRequest request, RequestOptions o * Asynchronously updates license for the cluster. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putLicenseAsync(PutLicenseRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, LicenseRequestConverters::putLicense, options, + public Cancellable putLicenseAsync(PutLicenseRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, LicenseRequestConverters::putLicense, options, PutLicenseResponse::fromXContent, listener, emptySet()); } @@ -101,9 +102,10 @@ public GetLicenseResponse getLicense(GetLicenseRequest request, RequestOptions o * Asynchronously returns the current license for the cluster cluster. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getLicenseAsync(GetLicenseRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsync(request, LicenseRequestConverters::getLicense, options, + public Cancellable getLicenseAsync(GetLicenseRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsync(request, LicenseRequestConverters::getLicense, options, response -> new GetLicenseResponse(convertResponseToJson(response)), listener, emptySet()); } @@ -122,9 +124,12 @@ public AcknowledgedResponse deleteLicense(DeleteLicenseRequest request, RequestO * Asynchronously deletes license from the cluster. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteLicenseAsync(DeleteLicenseRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, LicenseRequestConverters::deleteLicense, options, + public Cancellable deleteLicenseAsync(DeleteLicenseRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, + LicenseRequestConverters::deleteLicense, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -143,12 +148,13 @@ public StartTrialResponse startTrial(StartTrialRequest request, RequestOptions o * Asynchronously starts a trial license on the cluster. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void startTrialAsync(StartTrialRequest request, - RequestOptions options, - ActionListener listener) { + public Cancellable startTrialAsync(StartTrialRequest request, + RequestOptions options, + ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, LicenseRequestConverters::startTrial, options, + return restHighLevelClient.performRequestAsyncAndParseEntity(request, LicenseRequestConverters::startTrial, options, StartTrialResponse::fromXContent, listener, singleton(403)); } @@ -167,10 +173,11 @@ public StartBasicResponse startBasic(StartBasicRequest request, RequestOptions o * Asynchronously initiates an indefinite basic license. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void startBasicAsync(StartBasicRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, LicenseRequestConverters::startBasic, options, + public Cancellable startBasicAsync(StartBasicRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, LicenseRequestConverters::startBasic, options, StartBasicResponse::fromXContent, listener, emptySet()); } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java index ef078cf52dbd0..62619303685ae 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/MachineLearningClient.java @@ -153,13 +153,13 @@ public PutJobResponse putJob(PutJobRequest request, RequestOptions options) thro *

* For additional info * see ML PUT job documentation - * * @param request The request containing the {@link org.elasticsearch.client.ml.job.config.Job} settings * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putJobAsync(PutJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putJobAsync(PutJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::putJob, options, PutJobResponse::fromXContent, @@ -192,13 +192,13 @@ public GetJobResponse getJob(GetJobRequest request, RequestOptions options) thro *

* For additional info * see ML GET job documentation - * * @param request {@link GetJobRequest} Request containing a list of jobId(s) and additional options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified with {@link GetJobResponse} upon request completion + * @return cancellable that may be used to cancel the request */ - public void getJobAsync(GetJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getJobAsync(GetJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getJob, options, GetJobResponse::fromXContent, @@ -231,13 +231,13 @@ public GetJobStatsResponse getJobStats(GetJobStatsRequest request, RequestOption *

* For additional info * see Get job stats docs - * * @param request {@link GetJobStatsRequest} Request containing a list of jobId(s) and additional options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified with {@link GetJobStatsResponse} upon request completion + * @return cancellable that may be used to cancel the request */ - public void getJobStatsAsync(GetJobStatsRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getJobStatsAsync(GetJobStatsRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getJobStats, options, GetJobStatsResponse::fromXContent, @@ -272,14 +272,14 @@ public DeleteExpiredDataResponse deleteExpiredData(DeleteExpiredDataRequest requ * For additional info * see ML Delete Expired Data * documentation - * * @param request The request to delete expired ML data * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteExpiredDataAsync(DeleteExpiredDataRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteExpiredDataAsync(DeleteExpiredDataRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteExpiredData, options, DeleteExpiredDataResponse::fromXContent, @@ -313,12 +313,13 @@ public DeleteJobResponse deleteJob(DeleteJobRequest request, RequestOptions opti * For additional info * see ML Delete Job documentation * - * @param request The request to delete the job + * @param request The request to delete the job * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteJobAsync(DeleteJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteJobAsync(DeleteJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteJob, options, DeleteJobResponse::fromXContent, @@ -360,9 +361,10 @@ public OpenJobResponse openJob(OpenJobRequest request, RequestOptions options) t * @param request Request containing job_id and additional optional options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void openJobAsync(OpenJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable openJobAsync(OpenJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::openJob, options, OpenJobResponse::fromXContent, @@ -400,9 +402,10 @@ public CloseJobResponse closeJob(CloseJobRequest request, RequestOptions options * @param request Request containing job_ids and additional options. See {@link CloseJobRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void closeJobAsync(CloseJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable closeJobAsync(CloseJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::closeJob, options, CloseJobResponse::fromXContent, @@ -449,9 +452,10 @@ public FlushJobResponse flushJob(FlushJobRequest request, RequestOptions options * @param request The {@link FlushJobRequest} object enclosing the `jobId` and additional request options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void flushJobAsync(FlushJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable flushJobAsync(FlushJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::flushJob, options, FlushJobResponse::fromXContent, @@ -489,9 +493,10 @@ public ForecastJobResponse forecastJob(ForecastJobRequest request, RequestOption * @param request ForecastJobRequest with forecasting options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void forecastJobAsync(ForecastJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable forecastJobAsync(ForecastJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::forecastJob, options, ForecastJobResponse::fromXContent, @@ -529,9 +534,11 @@ public AcknowledgedResponse deleteForecast(DeleteForecastRequest request, Reques * @param request the {@link DeleteForecastRequest} object enclosing the desired jobId, forecastIDs, and other options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteForecastAsync(DeleteForecastRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteForecastAsync(DeleteForecastRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteForecast, options, AcknowledgedResponse::fromXContent, @@ -569,10 +576,11 @@ public AcknowledgedResponse deleteModelSnapshot(DeleteModelSnapshotRequest reque * @param request The request to delete the model snapshot * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteModelSnapshotAsync(DeleteModelSnapshotRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteModelSnapshotAsync(DeleteModelSnapshotRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteModelSnapshot, options, AcknowledgedResponse::fromXContent, @@ -610,10 +618,11 @@ public RevertModelSnapshotResponse revertModelSnapshot(RevertModelSnapshotReques * @param request The request to revert to a previous model snapshot * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void revertModelSnapshotAsync(RevertModelSnapshotRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable revertModelSnapshotAsync(RevertModelSnapshotRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::revertModelSnapshot, options, RevertModelSnapshotResponse::fromXContent, @@ -649,9 +658,10 @@ public PutDatafeedResponse putDatafeed(PutDatafeedRequest request, RequestOption * @param request The request containing the {@link org.elasticsearch.client.ml.datafeed.DatafeedConfig} settings * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putDatafeedAsync(PutDatafeedRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putDatafeedAsync(PutDatafeedRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::putDatafeed, options, PutDatafeedResponse::fromXContent, @@ -689,9 +699,11 @@ public PutDatafeedResponse updateDatafeed(UpdateDatafeedRequest request, Request * @param request The request containing the {@link org.elasticsearch.client.ml.datafeed.DatafeedUpdate} settings * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void updateDatafeedAsync(UpdateDatafeedRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable updateDatafeedAsync(UpdateDatafeedRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::updateDatafeed, options, PutDatafeedResponse::fromXContent, @@ -730,9 +742,11 @@ public GetDatafeedResponse getDatafeed(GetDatafeedRequest request, RequestOption * @param request {@link GetDatafeedRequest} Request containing a list of datafeedId(s) and additional options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified with {@link GetDatafeedResponse} upon request completion + * @return cancellable that may be used to cancel the request */ - public void getDatafeedAsync(GetDatafeedRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getDatafeedAsync(GetDatafeedRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getDatafeed, options, GetDatafeedResponse::fromXContent, @@ -770,9 +784,11 @@ public AcknowledgedResponse deleteDatafeed(DeleteDatafeedRequest request, Reques * @param request The request to delete the datafeed * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteDatafeedAsync(DeleteDatafeedRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteDatafeedAsync(DeleteDatafeedRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteDatafeed, options, AcknowledgedResponse::fromXContent, @@ -810,9 +826,11 @@ public StartDatafeedResponse startDatafeed(StartDatafeedRequest request, Request * @param request The request to start the datafeed * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void startDatafeedAsync(StartDatafeedRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable startDatafeedAsync(StartDatafeedRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::startDatafeed, options, StartDatafeedResponse::fromXContent, @@ -850,9 +868,11 @@ public StopDatafeedResponse stopDatafeed(StopDatafeedRequest request, RequestOpt * @param request The request to stop the datafeed * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void stopDatafeedAsync(StopDatafeedRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable stopDatafeedAsync(StopDatafeedRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::stopDatafeed, options, StopDatafeedResponse::fromXContent, @@ -910,11 +930,12 @@ public PreviewDatafeedResponse previewDatafeed(PreviewDatafeedRequest request, R * @param request {@link GetDatafeedStatsRequest} Request containing a list of datafeedId(s) and additional options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified with {@link GetDatafeedStatsResponse} upon request completion + * @return cancellable that may be used to cancel the request */ - public void getDatafeedStatsAsync(GetDatafeedStatsRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getDatafeedStatsAsync(GetDatafeedStatsRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getDatafeedStats, options, GetDatafeedStatsResponse::fromXContent, @@ -932,11 +953,12 @@ public void getDatafeedStatsAsync(GetDatafeedStatsRequest request, * @param request The request to preview the datafeed * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void previewDatafeedAsync(PreviewDatafeedRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable previewDatafeedAsync(PreviewDatafeedRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::previewDatafeed, options, PreviewDatafeedResponse::fromXContent, @@ -972,9 +994,10 @@ public PutJobResponse updateJob(UpdateJobRequest request, RequestOptions options * @param request the {@link UpdateJobRequest} object enclosing the desired updates * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void updateJobAsync(UpdateJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable updateJobAsync(UpdateJobRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::updateJob, options, PutJobResponse::fromXContent, @@ -1005,12 +1028,13 @@ public GetBucketsResponse getBuckets(GetBucketsRequest request, RequestOptions o * For additional info * see ML GET buckets documentation * - * @param request The request + * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getBucketsAsync(GetBucketsRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getBucketsAsync(GetBucketsRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getBuckets, options, GetBucketsResponse::fromXContent, @@ -1047,9 +1071,11 @@ public GetCategoriesResponse getCategories(GetCategoriesRequest request, Request * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getCategoriesAsync(GetCategoriesRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getCategoriesAsync(GetCategoriesRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getCategories, options, GetCategoriesResponse::fromXContent, @@ -1086,10 +1112,11 @@ public GetModelSnapshotsResponse getModelSnapshots(GetModelSnapshotsRequest requ * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getModelSnapshotsAsync(GetModelSnapshotsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getModelSnapshotsAsync(GetModelSnapshotsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getModelSnapshots, options, GetModelSnapshotsResponse::fromXContent, @@ -1127,10 +1154,11 @@ public UpdateModelSnapshotResponse updateModelSnapshot(UpdateModelSnapshotReques * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void updateModelSnapshotAsync(UpdateModelSnapshotRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable updateModelSnapshotAsync(UpdateModelSnapshotRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::updateModelSnapshot, options, UpdateModelSnapshotResponse::fromXContent, @@ -1166,10 +1194,11 @@ public GetOverallBucketsResponse getOverallBuckets(GetOverallBucketsRequest requ * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getOverallBucketsAsync(GetOverallBucketsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getOverallBucketsAsync(GetOverallBucketsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getOverallBuckets, options, GetOverallBucketsResponse::fromXContent, @@ -1203,9 +1232,10 @@ public GetRecordsResponse getRecords(GetRecordsRequest request, RequestOptions o * @param request the request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getRecordsAsync(GetRecordsRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getRecordsAsync(GetRecordsRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getRecords, options, GetRecordsResponse::fromXContent, @@ -1245,9 +1275,10 @@ public PostDataResponse postData(PostDataRequest request, RequestOptions options * @param request PostDataRequest containing the data to post and some additional options * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void postDataAsync(PostDataRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable postDataAsync(PostDataRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::postData, options, PostDataResponse::fromXContent, @@ -1283,9 +1314,11 @@ public GetCalendarsResponse getCalendars(GetCalendarsRequest request, RequestOpt * @param request The calendars request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getCalendarsAsync(GetCalendarsRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getCalendarsAsync(GetCalendarsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getCalendars, options, GetCalendarsResponse::fromXContent, @@ -1321,10 +1354,11 @@ public GetInfluencersResponse getInfluencers(GetInfluencersRequest request, Requ * @param request the request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getInfluencersAsync(GetInfluencersRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getInfluencersAsync(GetInfluencersRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getInfluencers, options, GetInfluencersResponse::fromXContent, @@ -1362,9 +1396,10 @@ public PutCalendarResponse putCalendar(PutCalendarRequest request, RequestOption * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putCalendarAsync(PutCalendarRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putCalendarAsync(PutCalendarRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::putCalendar, options, PutCalendarResponse::fromXContent, @@ -1402,9 +1437,11 @@ public PutCalendarResponse putCalendarJob(PutCalendarJobRequest request, Request * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putCalendarJobAsync(PutCalendarJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putCalendarJobAsync(PutCalendarJobRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::putCalendarJob, options, PutCalendarResponse::fromXContent, @@ -1442,11 +1479,12 @@ public PutCalendarResponse deleteCalendarJob(DeleteCalendarJobRequest request, R * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteCalendarJobAsync(DeleteCalendarJobRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteCalendarJobAsync(DeleteCalendarJobRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteCalendarJob, options, PutCalendarResponse::fromXContent, @@ -1484,9 +1522,11 @@ public AcknowledgedResponse deleteCalendar(DeleteCalendarRequest request, Reques * @param request The request to delete the calendar * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteCalendarAsync(DeleteCalendarRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteCalendarAsync(DeleteCalendarRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteCalendar, options, AcknowledgedResponse::fromXContent, @@ -1524,10 +1564,11 @@ public GetCalendarEventsResponse getCalendarEvents(GetCalendarEventsRequest requ * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getCalendarEventsAsync(GetCalendarEventsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getCalendarEventsAsync(GetCalendarEventsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getCalendarEvents, options, GetCalendarEventsResponse::fromXContent, @@ -1565,10 +1606,11 @@ public PostCalendarEventResponse postCalendarEvent(PostCalendarEventRequest requ * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void postCalendarEventAsync(PostCalendarEventRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable postCalendarEventAsync(PostCalendarEventRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::postCalendarEvents, options, PostCalendarEventResponse::fromXContent, @@ -1606,11 +1648,12 @@ public AcknowledgedResponse deleteCalendarEvent(DeleteCalendarEventRequest reque * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteCalendarEventAsync(DeleteCalendarEventRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteCalendarEventAsync(DeleteCalendarEventRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteCalendarEvent, options, AcknowledgedResponse::fromXContent, @@ -1646,9 +1689,10 @@ public PutFilterResponse putFilter(PutFilterRequest request, RequestOptions opti * @param request The request containing the {@link org.elasticsearch.client.ml.job.config.MlFilter} settings * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putFilterAsync(PutFilterRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putFilterAsync(PutFilterRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::putFilter, options, PutFilterResponse::fromXContent, @@ -1684,9 +1728,10 @@ public GetFiltersResponse getFilter(GetFiltersRequest request, RequestOptions op * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getFilterAsync(GetFiltersRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getFilterAsync(GetFiltersRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getFilter, options, GetFiltersResponse::fromXContent, @@ -1724,9 +1769,10 @@ public PutFilterResponse updateFilter(UpdateFilterRequest request, RequestOption * @param request The request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void updateFilterAsync(UpdateFilterRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable updateFilterAsync(UpdateFilterRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::updateFilter, options, PutFilterResponse::fromXContent, @@ -1764,9 +1810,11 @@ public AcknowledgedResponse deleteFilter(DeleteFilterRequest request, RequestOpt * @param request The request to delete the filter * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteFilterAsync(DeleteFilterRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteFilterAsync(DeleteFilterRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteFilter, options, AcknowledgedResponse::fromXContent, @@ -1802,9 +1850,10 @@ public MlInfoResponse getMlInfo(MlInfoRequest request, RequestOptions options) t * @param request The request of Machine Learning info * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getMlInfoAsync(MlInfoRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getMlInfoAsync(MlInfoRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::mlInfo, options, MlInfoResponse::fromXContent, @@ -1842,10 +1891,11 @@ public FindFileStructureResponse findFileStructure(FindFileStructureRequest requ * @param request The find file structure request * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void findFileStructureAsync(FindFileStructureRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable findFileStructureAsync(FindFileStructureRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::findFileStructure, options, FindFileStructureResponse::fromXContent, @@ -1881,9 +1931,11 @@ public AcknowledgedResponse setUpgradeMode(SetUpgradeModeRequest request, Reques * @param request The request of Machine Learning info * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void setUpgradeModeAsync(SetUpgradeModeRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable setUpgradeModeAsync(SetUpgradeModeRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::setUpgradeMode, options, AcknowledgedResponse::fromXContent, @@ -1925,10 +1977,11 @@ public PutDataFrameAnalyticsResponse putDataFrameAnalytics(PutDataFrameAnalytics * {@link org.elasticsearch.client.ml.dataframe.DataFrameAnalyticsConfig} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putDataFrameAnalyticsAsync(PutDataFrameAnalyticsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putDataFrameAnalyticsAsync(PutDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::putDataFrameAnalytics, options, PutDataFrameAnalyticsResponse::fromXContent, @@ -1967,10 +2020,11 @@ public GetDataFrameAnalyticsResponse getDataFrameAnalytics(GetDataFrameAnalytics * @param request The {@link GetDataFrameAnalyticsRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getDataFrameAnalyticsAsync(GetDataFrameAnalyticsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getDataFrameAnalyticsAsync(GetDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getDataFrameAnalytics, options, GetDataFrameAnalyticsResponse::fromXContent, @@ -2008,10 +2062,11 @@ public GetDataFrameAnalyticsStatsResponse getDataFrameAnalyticsStats(GetDataFram * @param request The {@link GetDataFrameAnalyticsStatsRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getDataFrameAnalyticsStatsAsync(GetDataFrameAnalyticsStatsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getDataFrameAnalyticsStatsAsync(GetDataFrameAnalyticsStatsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::getDataFrameAnalyticsStats, options, GetDataFrameAnalyticsStatsResponse::fromXContent, @@ -2047,13 +2102,14 @@ public AcknowledgedResponse startDataFrameAnalytics(StartDataFrameAnalyticsReque * see * Start Data Frame Analytics documentation * - * @param request The {@link StartDataFrameAnalyticsRequest} + * @param request The {@link StartDataFrameAnalyticsRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void startDataFrameAnalyticsAsync(StartDataFrameAnalyticsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable startDataFrameAnalyticsAsync(StartDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::startDataFrameAnalytics, options, AcknowledgedResponse::fromXContent, @@ -2092,10 +2148,11 @@ public StopDataFrameAnalyticsResponse stopDataFrameAnalytics(StopDataFrameAnalyt * @param request The {@link StopDataFrameAnalyticsRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void stopDataFrameAnalyticsAsync(StopDataFrameAnalyticsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable stopDataFrameAnalyticsAsync(StopDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::stopDataFrameAnalytics, options, StopDataFrameAnalyticsResponse::fromXContent, @@ -2134,10 +2191,11 @@ public AcknowledgedResponse deleteDataFrameAnalytics(DeleteDataFrameAnalyticsReq * @param request The {@link DeleteDataFrameAnalyticsRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteDataFrameAnalyticsAsync(DeleteDataFrameAnalyticsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteDataFrameAnalyticsAsync(DeleteDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::deleteDataFrameAnalytics, options, AcknowledgedResponse::fromXContent, @@ -2176,10 +2234,11 @@ public EvaluateDataFrameResponse evaluateDataFrame(EvaluateDataFrameRequest requ * @param request The {@link EvaluateDataFrameRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void evaluateDataFrameAsync(EvaluateDataFrameRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable evaluateDataFrameAsync(EvaluateDataFrameRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MLRequestConverters::evaluateDataFrame, options, EvaluateDataFrameResponse::fromXContent, @@ -2219,10 +2278,11 @@ public EstimateMemoryUsageResponse estimateMemoryUsage(PutDataFrameAnalyticsRequ * @param request The {@link PutDataFrameAnalyticsRequest} * @param options Additional request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener Listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void estimateMemoryUsageAsync(PutDataFrameAnalyticsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable estimateMemoryUsageAsync(PutDataFrameAnalyticsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, MLRequestConverters::estimateMemoryUsage, options, diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/MigrationClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/MigrationClient.java index a95115f71faa8..32f7cb140e5c5 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/MigrationClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/MigrationClient.java @@ -58,10 +58,11 @@ public DeprecationInfoResponse getDeprecationInfo(DeprecationInfoRequest request * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getDeprecationInfoAsync(DeprecationInfoRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, MigrationRequestConverters::getDeprecationInfo, options, + public Cancellable getDeprecationInfoAsync(DeprecationInfoRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, MigrationRequestConverters::getDeprecationInfo, options, DeprecationInfoResponse::fromXContent, listener, Collections.emptySet()); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java index 4e1a571ec6560..d3c118ad0ae0c 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java @@ -498,9 +498,11 @@ public final BulkResponse bulk(BulkRequest bulkRequest, RequestOptions options) * @param bulkRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void bulkAsync(BulkRequest bulkRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(bulkRequest, RequestConverters::bulk, options, BulkResponse::fromXContent, listener, emptySet()); + public final Cancellable bulkAsync(BulkRequest bulkRequest, RequestOptions options, ActionListener listener) { + return performRequestAsyncAndParseEntity(bulkRequest, RequestConverters::bulk, options, + BulkResponse::fromXContent, listener, emptySet()); } /** @@ -535,9 +537,11 @@ public final TaskSubmissionResponse submitReindexTask(ReindexRequest reindexRequ * @param reindexRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void reindexAsync(ReindexRequest reindexRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity( + public final Cancellable reindexAsync(ReindexRequest reindexRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity( reindexRequest, RequestConverters::reindex, options, BulkByScrollResponse::fromXContent, listener, singleton(409) ); } @@ -563,10 +567,11 @@ public final BulkByScrollResponse updateByQuery(UpdateByQueryRequest updateByQue * @param updateByQueryRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void updateByQueryAsync(UpdateByQueryRequest updateByQueryRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity( + public final Cancellable updateByQueryAsync(UpdateByQueryRequest updateByQueryRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity( updateByQueryRequest, RequestConverters::updateByQuery, options, BulkByScrollResponse::fromXContent, listener, singleton(409) ); } @@ -592,10 +597,11 @@ public final BulkByScrollResponse deleteByQuery(DeleteByQueryRequest deleteByQue * @param deleteByQueryRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void deleteByQueryAsync(DeleteByQueryRequest deleteByQueryRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity( + public final Cancellable deleteByQueryAsync(DeleteByQueryRequest deleteByQueryRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity( deleteByQueryRequest, RequestConverters::deleteByQuery, options, BulkByScrollResponse::fromXContent, listener, singleton(409) ); } @@ -620,10 +626,11 @@ public final ListTasksResponse deleteByQueryRethrottle(RethrottleRequest rethrot * @param rethrottleRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void deleteByQueryRethrottleAsync(RethrottleRequest rethrottleRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(rethrottleRequest, RequestConverters::rethrottleDeleteByQuery, options, + public final Cancellable deleteByQueryRethrottleAsync(RethrottleRequest rethrottleRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(rethrottleRequest, RequestConverters::rethrottleDeleteByQuery, options, ListTasksResponse::fromXContent, listener, emptySet()); } @@ -647,10 +654,11 @@ public final ListTasksResponse updateByQueryRethrottle(RethrottleRequest rethrot * @param rethrottleRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void updateByQueryRethrottleAsync(RethrottleRequest rethrottleRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(rethrottleRequest, RequestConverters::rethrottleUpdateByQuery, options, + public final Cancellable updateByQueryRethrottleAsync(RethrottleRequest rethrottleRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(rethrottleRequest, RequestConverters::rethrottleUpdateByQuery, options, ListTasksResponse::fromXContent, listener, emptySet()); } @@ -672,15 +680,15 @@ public final ListTasksResponse reindexRethrottle(RethrottleRequest rethrottleReq * Executes a reindex rethrottling request. * See the * Reindex rethrottling API on elastic.co - * * @param rethrottleRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void reindexRethrottleAsync(RethrottleRequest rethrottleRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(rethrottleRequest, RequestConverters::rethrottleReindex, options, ListTasksResponse::fromXContent, - listener, emptySet()); + public final Cancellable reindexRethrottleAsync(RethrottleRequest rethrottleRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(rethrottleRequest, + RequestConverters::rethrottleReindex, options, ListTasksResponse::fromXContent, listener, emptySet()); } /** @@ -720,9 +728,10 @@ public final GetResponse get(GetRequest getRequest, RequestOptions options) thro * @param getRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void getAsync(GetRequest getRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(getRequest, RequestConverters::get, options, GetResponse::fromXContent, listener, + public final Cancellable getAsync(GetRequest getRequest, RequestOptions options, ActionListener listener) { + return performRequestAsyncAndParseEntity(getRequest, RequestConverters::get, options, GetResponse::fromXContent, listener, singleton(404)); } @@ -759,10 +768,12 @@ public final MultiGetResponse mget(MultiGetRequest multiGetRequest, RequestOptio * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion * @deprecated use {@link #mgetAsync(MultiGetRequest, RequestOptions, ActionListener)} instead + * @return cancellable that may be used to cancel the request */ @Deprecated - public final void multiGetAsync(MultiGetRequest multiGetRequest, RequestOptions options, ActionListener listener) { - mgetAsync(multiGetRequest, options, listener); + public final Cancellable multiGetAsync(MultiGetRequest multiGetRequest, RequestOptions options, + ActionListener listener) { + return mgetAsync(multiGetRequest, options, listener); } /** @@ -771,10 +782,12 @@ public final void multiGetAsync(MultiGetRequest multiGetRequest, RequestOptions * @param multiGetRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void mgetAsync(MultiGetRequest multiGetRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(multiGetRequest, RequestConverters::multiGet, options, MultiGetResponse::fromXContent, listener, - singleton(404)); + public final Cancellable mgetAsync(MultiGetRequest multiGetRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(multiGetRequest, RequestConverters::multiGet, options, + MultiGetResponse::fromXContent, listener, singleton(404)); } /** @@ -794,9 +807,10 @@ public final boolean exists(GetRequest getRequest, RequestOptions options) throw * @param getRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void existsAsync(GetRequest getRequest, RequestOptions options, ActionListener listener) { - performRequestAsync(getRequest, RequestConverters::exists, options, RestHighLevelClient::convertExistsResponse, listener, + public final Cancellable existsAsync(GetRequest getRequest, RequestOptions options, ActionListener listener) { + return performRequestAsync(getRequest, RequestConverters::exists, options, RestHighLevelClient::convertExistsResponse, listener, emptySet()); } @@ -819,10 +833,11 @@ public boolean existsSource(GetRequest getRequest, RequestOptions options) throw * @param getRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void existsSourceAsync(GetRequest getRequest, RequestOptions options, ActionListener listener) { - performRequestAsync(getRequest, RequestConverters::sourceExists, options, RestHighLevelClient::convertExistsResponse, listener, - emptySet()); + public final Cancellable existsSourceAsync(GetRequest getRequest, RequestOptions options, ActionListener listener) { + return performRequestAsync(getRequest, RequestConverters::sourceExists, options, + RestHighLevelClient::convertExistsResponse, listener, emptySet()); } /** @@ -833,7 +848,8 @@ public final void existsSourceAsync(GetRequest getRequest, RequestOptions option * @return the response */ public final IndexResponse index(IndexRequest indexRequest, RequestOptions options) throws IOException { - return performRequestAndParseEntity(indexRequest, RequestConverters::index, options, IndexResponse::fromXContent, emptySet()); + return performRequestAndParseEntity(indexRequest, RequestConverters::index, options, + IndexResponse::fromXContent, emptySet()); } /** @@ -842,9 +858,10 @@ public final IndexResponse index(IndexRequest indexRequest, RequestOptions optio * @param indexRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void indexAsync(IndexRequest indexRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(indexRequest, RequestConverters::index, options, IndexResponse::fromXContent, listener, + public final Cancellable indexAsync(IndexRequest indexRequest, RequestOptions options, ActionListener listener) { + return performRequestAsyncAndParseEntity(indexRequest, RequestConverters::index, options, IndexResponse::fromXContent, listener, emptySet()); } @@ -866,9 +883,10 @@ public final CountResponse count(CountRequest countRequest, RequestOptions optio * @param countRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void countAsync(CountRequest countRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(countRequest, RequestConverters::count, options,CountResponse::fromXContent, + public final Cancellable countAsync(CountRequest countRequest, RequestOptions options, ActionListener listener) { + return performRequestAsyncAndParseEntity(countRequest, RequestConverters::count, options,CountResponse::fromXContent, listener, emptySet()); } @@ -889,9 +907,10 @@ public final UpdateResponse update(UpdateRequest updateRequest, RequestOptions o * @param updateRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void updateAsync(UpdateRequest updateRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(updateRequest, RequestConverters::update, options, UpdateResponse::fromXContent, listener, + public final Cancellable updateAsync(UpdateRequest updateRequest, RequestOptions options, ActionListener listener) { + return performRequestAsyncAndParseEntity(updateRequest, RequestConverters::update, options, UpdateResponse::fromXContent, listener, emptySet()); } @@ -913,9 +932,10 @@ public final DeleteResponse delete(DeleteRequest deleteRequest, RequestOptions o * @param deleteRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void deleteAsync(DeleteRequest deleteRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(deleteRequest, RequestConverters::delete, options, DeleteResponse::fromXContent, listener, + public final Cancellable deleteAsync(DeleteRequest deleteRequest, RequestOptions options, ActionListener listener) { + return performRequestAsyncAndParseEntity(deleteRequest, RequestConverters::delete, options, DeleteResponse::fromXContent, listener, Collections.singleton(404)); } @@ -941,9 +961,10 @@ public final SearchResponse search(SearchRequest searchRequest, RequestOptions o * @param searchRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void searchAsync(SearchRequest searchRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity( + public final Cancellable searchAsync(SearchRequest searchRequest, RequestOptions options, ActionListener listener) { + return performRequestAsyncAndParseEntity( searchRequest, r -> RequestConverters.search(r, "_search"), options, @@ -987,11 +1008,12 @@ public final MultiSearchResponse msearch(MultiSearchRequest multiSearchRequest, * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion * @deprecated use {@link #msearchAsync(MultiSearchRequest, RequestOptions, ActionListener)} instead + * @return cancellable that may be used to cancel the request */ @Deprecated - public final void multiSearchAsync(MultiSearchRequest searchRequest, RequestOptions options, - ActionListener listener) { - msearchAsync(searchRequest, options, listener); + public final Cancellable multiSearchAsync(MultiSearchRequest searchRequest, RequestOptions options, + ActionListener listener) { + return msearchAsync(searchRequest, options, listener); } /** @@ -1001,10 +1023,11 @@ public final void multiSearchAsync(MultiSearchRequest searchRequest, RequestOpti * @param searchRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void msearchAsync(MultiSearchRequest searchRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(searchRequest, RequestConverters::multiSearch, options, MultiSearchResponse::fromXContext, + public final Cancellable msearchAsync(MultiSearchRequest searchRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(searchRequest, RequestConverters::multiSearch, options, MultiSearchResponse::fromXContext, listener, emptySet()); } @@ -1046,11 +1069,12 @@ public final SearchResponse scroll(SearchScrollRequest searchScrollRequest, Requ * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion * @deprecated use {@link #scrollAsync(SearchScrollRequest, RequestOptions, ActionListener)} instead + * @return cancellable that may be used to cancel the request */ @Deprecated - public final void searchScrollAsync(SearchScrollRequest searchScrollRequest, RequestOptions options, - ActionListener listener) { - scrollAsync(searchScrollRequest, options, listener); + public final Cancellable searchScrollAsync(SearchScrollRequest searchScrollRequest, RequestOptions options, + ActionListener listener) { + return scrollAsync(searchScrollRequest, options, listener); } /** @@ -1061,11 +1085,12 @@ public final void searchScrollAsync(SearchScrollRequest searchScrollRequest, Req * @param searchScrollRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void scrollAsync(SearchScrollRequest searchScrollRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(searchScrollRequest, RequestConverters::searchScroll, options, SearchResponse::fromXContent, - listener, emptySet()); + public final Cancellable scrollAsync(SearchScrollRequest searchScrollRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(searchScrollRequest, RequestConverters::searchScroll, + options, SearchResponse::fromXContent, listener, emptySet()); } /** @@ -1090,11 +1115,12 @@ public final ClearScrollResponse clearScroll(ClearScrollRequest clearScrollReque * @param clearScrollRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void clearScrollAsync(ClearScrollRequest clearScrollRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(clearScrollRequest, RequestConverters::clearScroll, options, ClearScrollResponse::fromXContent, - listener, emptySet()); + public final Cancellable clearScrollAsync(ClearScrollRequest clearScrollRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(clearScrollRequest, RequestConverters::clearScroll, + options, ClearScrollResponse::fromXContent, listener, emptySet()); } /** @@ -1116,10 +1142,11 @@ public final SearchTemplateResponse searchTemplate(SearchTemplateRequest searchT * * See Search Template API * on elastic.co. + * @return cancellable that may be used to cancel the request */ - public final void searchTemplateAsync(SearchTemplateRequest searchTemplateRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(searchTemplateRequest, RequestConverters::searchTemplate, options, + public final Cancellable searchTemplateAsync(SearchTemplateRequest searchTemplateRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(searchTemplateRequest, RequestConverters::searchTemplate, options, SearchTemplateResponse::fromXContent, listener, emptySet()); } @@ -1147,9 +1174,10 @@ public final ExplainResponse explain(ExplainRequest explainRequest, RequestOptio * @param explainRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void explainAsync(ExplainRequest explainRequest, RequestOptions options, ActionListener listener) { - performRequestAsync(explainRequest, RequestConverters::explain, options, + public final Cancellable explainAsync(ExplainRequest explainRequest, RequestOptions options, ActionListener listener) { + return performRequestAsync(explainRequest, RequestConverters::explain, options, response -> { CheckedFunction entityParser = parser -> ExplainResponse.fromXContent(parser, convertExistsResponse(response)); @@ -1181,9 +1209,12 @@ public final TermVectorsResponse termvectors(TermVectorsRequest request, Request * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void termvectorsAsync(TermVectorsRequest request, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(request, RequestConverters::termVectors, options, TermVectorsResponse::fromXContent, listener, + public final Cancellable termvectorsAsync(TermVectorsRequest request, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(request, RequestConverters::termVectors, options, + TermVectorsResponse::fromXContent, listener, emptySet()); } @@ -1211,10 +1242,11 @@ public final MultiTermVectorsResponse mtermvectors(MultiTermVectorsRequest reque * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void mtermvectorsAsync(MultiTermVectorsRequest request, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity( + public final Cancellable mtermvectorsAsync(MultiTermVectorsRequest request, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity( request, RequestConverters::mtermVectors, options, MultiTermVectorsResponse::fromXContent, listener, emptySet()); } @@ -1250,11 +1282,12 @@ public final MultiSearchTemplateResponse msearchTemplate(MultiSearchTemplateRequ * * See Multi Search Template API * on elastic.co. + * @return cancellable that may be used to cancel the request */ - public final void msearchTemplateAsync(MultiSearchTemplateRequest multiSearchTemplateRequest, - RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(multiSearchTemplateRequest, RequestConverters::multiSearchTemplate, + public final Cancellable msearchTemplateAsync(MultiSearchTemplateRequest multiSearchTemplateRequest, + RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(multiSearchTemplateRequest, RequestConverters::multiSearchTemplate, options, MultiSearchTemplateResponse::fromXContext, listener, emptySet()); } @@ -1265,9 +1298,12 @@ public final void msearchTemplateAsync(MultiSearchTemplateRequest multiSearchTem * @param rankEvalRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void rankEvalAsync(RankEvalRequest rankEvalRequest, RequestOptions options, ActionListener listener) { - performRequestAsyncAndParseEntity(rankEvalRequest, RequestConverters::rankEval, options, RankEvalResponse::fromXContent, listener, + public final Cancellable rankEvalAsync(RankEvalRequest rankEvalRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(rankEvalRequest, RequestConverters::rankEval, options, + RankEvalResponse::fromXContent, listener, emptySet()); } @@ -1305,10 +1341,11 @@ public GetStoredScriptResponse getScript(GetStoredScriptRequest request, Request * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getScriptAsync(GetStoredScriptRequest request, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(request, RequestConverters::getScript, options, + public Cancellable getScriptAsync(GetStoredScriptRequest request, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(request, RequestConverters::getScript, options, GetStoredScriptResponse::fromXContent, listener, emptySet()); } @@ -1332,10 +1369,11 @@ public AcknowledgedResponse deleteScript(DeleteStoredScriptRequest request, Requ * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteScriptAsync(DeleteStoredScriptRequest request, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(request, RequestConverters::deleteScript, options, + public Cancellable deleteScriptAsync(DeleteStoredScriptRequest request, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(request, RequestConverters::deleteScript, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1360,10 +1398,11 @@ public AcknowledgedResponse putScript(PutStoredScriptRequest putStoredScriptRequ * @param putStoredScriptRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putScriptAsync(PutStoredScriptRequest putStoredScriptRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(putStoredScriptRequest, RequestConverters::putScript, options, + public Cancellable putScriptAsync(PutStoredScriptRequest putStoredScriptRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(putStoredScriptRequest, RequestConverters::putScript, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -1374,10 +1413,11 @@ public void putScriptAsync(PutStoredScriptRequest putStoredScriptRequest, Reques * @param fieldCapabilitiesRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public final void fieldCapsAsync(FieldCapabilitiesRequest fieldCapabilitiesRequest, RequestOptions options, - ActionListener listener) { - performRequestAsyncAndParseEntity(fieldCapabilitiesRequest, RequestConverters::fieldCaps, options, + public final Cancellable fieldCapsAsync(FieldCapabilitiesRequest fieldCapabilitiesRequest, RequestOptions options, + ActionListener listener) { + return performRequestAsyncAndParseEntity(fieldCapabilitiesRequest, RequestConverters::fieldCaps, options, FieldCapabilitiesResponse::fromXContent, listener, emptySet()); } @@ -1509,26 +1549,28 @@ protected final Optional performRequestAnd /** * @deprecated If creating a new HLRC ReST API call, consider creating new actions instead of reusing server actions. The Validation * layer has been added to the ReST client, and requests should extend {@link Validatable} instead of {@link ActionRequest}. + * @return Cancellable instance that may be used to cancel the request */ @Deprecated - protected final void performRequestAsyncAndParseEntity(Req request, - CheckedFunction requestConverter, - RequestOptions options, - CheckedFunction entityParser, - ActionListener listener, Set ignores) { - performRequestAsync(request, requestConverter, options, + protected final Cancellable performRequestAsyncAndParseEntity(Req request, + CheckedFunction requestConverter, + RequestOptions options, + CheckedFunction entityParser, + ActionListener listener, Set ignores) { + return performRequestAsync(request, requestConverter, options, response -> parseEntity(response.getEntity(), entityParser), listener, ignores); } /** * Defines a helper method for asynchronously performing a request. - */ - protected final void performRequestAsyncAndParseEntity(Req request, - CheckedFunction requestConverter, - RequestOptions options, - CheckedFunction entityParser, - ActionListener listener, Set ignores) { - performRequestAsync(request, requestConverter, options, + * @return Cancellable instance that may be used to cancel the request + */ + protected final Cancellable performRequestAsyncAndParseEntity(Req request, + CheckedFunction requestConverter, + RequestOptions options, + CheckedFunction entityParser, + ActionListener listener, Set ignores) { + return performRequestAsync(request, requestConverter, options, response -> parseEntity(response.getEntity(), entityParser), listener, ignores); } @@ -1536,56 +1578,59 @@ protected final void performRequestAsyncAndParse /** * @deprecated If creating a new HLRC ReST API call, consider creating new actions instead of reusing server actions. The Validation * layer has been added to the ReST client, and requests should extend {@link Validatable} instead of {@link ActionRequest}. + * @return Cancellable instance that may be used to cancel the request */ @Deprecated - protected final void performRequestAsync(Req request, - CheckedFunction requestConverter, - RequestOptions options, - CheckedFunction responseConverter, - ActionListener listener, Set ignores) { + protected final Cancellable performRequestAsync(Req request, + CheckedFunction requestConverter, + RequestOptions options, + CheckedFunction responseConverter, + ActionListener listener, Set ignores) { ActionRequestValidationException validationException = request.validate(); if (validationException != null && validationException.validationErrors().isEmpty() == false) { listener.onFailure(validationException); - return; + return Cancellable.NO_OP; } - internalPerformRequestAsync(request, requestConverter, options, responseConverter, listener, ignores); + return internalPerformRequestAsync(request, requestConverter, options, responseConverter, listener, ignores); } /** * Defines a helper method for asynchronously performing a request. + * @return Cancellable instance that may be used to cancel the request */ - protected final void performRequestAsync(Req request, - CheckedFunction requestConverter, - RequestOptions options, - CheckedFunction responseConverter, - ActionListener listener, Set ignores) { + protected final Cancellable performRequestAsync(Req request, + CheckedFunction requestConverter, + RequestOptions options, + CheckedFunction responseConverter, + ActionListener listener, Set ignores) { Optional validationException = request.validate(); if (validationException != null && validationException.isPresent()) { listener.onFailure(validationException.get()); - return; + return Cancellable.NO_OP; } - internalPerformRequestAsync(request, requestConverter, options, responseConverter, listener, ignores); + return internalPerformRequestAsync(request, requestConverter, options, responseConverter, listener, ignores); } /** * Provides common functionality for asynchronously performing a request. + * @return Cancellable instance that may be used to cancel the request */ - private void internalPerformRequestAsync(Req request, - CheckedFunction requestConverter, - RequestOptions options, - CheckedFunction responseConverter, - ActionListener listener, Set ignores) { + private Cancellable internalPerformRequestAsync(Req request, + CheckedFunction requestConverter, + RequestOptions options, + CheckedFunction responseConverter, + ActionListener listener, Set ignores) { Request req; try { req = requestConverter.apply(request); } catch (Exception e) { listener.onFailure(e); - return; + return Cancellable.NO_OP; } req.setOptions(options); ResponseListener responseListener = wrapResponseListener(responseConverter, listener, ignores); - client.performRequestAsync(req, responseListener); + return client.performRequestAsync(req, responseListener); } @@ -1629,28 +1674,29 @@ public void onFailure(Exception exception) { /** * Asynchronous request which returns empty {@link Optional}s in the case of 404s or parses entity into an Optional + * @return Cancellable instance that may be used to cancel the request */ - protected final void performRequestAsyncAndParseOptionalEntity(Req request, - CheckedFunction requestConverter, - RequestOptions options, - CheckedFunction entityParser, - ActionListener> listener) { + protected final Cancellable performRequestAsyncAndParseOptionalEntity(Req request, + CheckedFunction requestConverter, + RequestOptions options, + CheckedFunction entityParser, + ActionListener> listener) { Optional validationException = request.validate(); if (validationException != null && validationException.isPresent()) { listener.onFailure(validationException.get()); - return; + return Cancellable.NO_OP; } Request req; try { req = requestConverter.apply(request); } catch (Exception e) { listener.onFailure(e); - return; + return Cancellable.NO_OP; } req.setOptions(options); ResponseListener responseListener = wrapResponseListener404sOptional(response -> parseEntity(response.getEntity(), entityParser), listener); - client.performRequestAsync(req, responseListener); + return client.performRequestAsync(req, responseListener); } final ResponseListener wrapResponseListener404sOptional(CheckedFunction responseConverter, diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupClient.java index cf0ceec70b52e..55dabc2be0e62 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RollupClient.java @@ -30,8 +30,6 @@ import org.elasticsearch.client.rollup.GetRollupJobResponse; import org.elasticsearch.client.rollup.GetRollupCapsRequest; import org.elasticsearch.client.rollup.GetRollupCapsResponse; -import org.elasticsearch.client.rollup.GetRollupJobRequest; -import org.elasticsearch.client.rollup.GetRollupJobResponse; import org.elasticsearch.client.rollup.PutRollupJobRequest; import org.elasticsearch.client.rollup.StartRollupJobRequest; import org.elasticsearch.client.rollup.StartRollupJobResponse; @@ -80,9 +78,11 @@ public AcknowledgedResponse putRollupJob(PutRollupJobRequest request, RequestOpt * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putRollupJobAsync(PutRollupJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable putRollupJobAsync(PutRollupJobRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, RollupRequestConverters::putJob, options, AcknowledgedResponse::fromXContent, @@ -113,10 +113,11 @@ public StartRollupJobResponse startRollupJob(StartRollupJobRequest request, Requ * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void startRollupJobAsync(StartRollupJobRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable startRollupJobAsync(StartRollupJobRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, RollupRequestConverters::startJob, options, StartRollupJobResponse::fromXContent, @@ -147,10 +148,11 @@ public StopRollupJobResponse stopRollupJob(StopRollupJobRequest request, Request * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void stopRollupJobAsync(StopRollupJobRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable stopRollupJobAsync(StopRollupJobRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, RollupRequestConverters::stopJob, options, StopRollupJobResponse::fromXContent, @@ -180,11 +182,12 @@ public AcknowledgedResponse deleteRollupJob(DeleteRollupJobRequest request, Requ * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteRollupJobAsync(DeleteRollupJobRequest request, - RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable deleteRollupJobAsync(DeleteRollupJobRequest request, + RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, RollupRequestConverters::deleteJob, options, AcknowledgedResponse::fromXContent, @@ -215,11 +218,11 @@ public GetRollupJobResponse getRollupJob(GetRollupJobRequest request, RequestOpt * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - - - public void getRollupJobAsync(GetRollupJobRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getRollupJobAsync(GetRollupJobRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, RollupRequestConverters::getJob, options, GetRollupJobResponse::fromXContent, @@ -251,9 +254,10 @@ public SearchResponse search(SearchRequest request, RequestOptions options) thro * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void searchAsync(SearchRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable searchAsync(SearchRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, RollupRequestConverters::search, options, @@ -286,10 +290,11 @@ public GetRollupCapsResponse getRollupCapabilities(GetRollupCapsRequest request, * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getRollupCapabilitiesAsync(GetRollupCapsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getRollupCapabilitiesAsync(GetRollupCapsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, RollupRequestConverters::getRollupCaps, options, GetRollupCapsResponse::fromXContent, @@ -322,10 +327,11 @@ public GetRollupIndexCapsResponse getRollupIndexCapabilities(GetRollupIndexCapsR * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getRollupIndexCapabilitiesAsync(GetRollupIndexCapsRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, + public Cancellable getRollupIndexCapabilitiesAsync(GetRollupIndexCapsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, RollupRequestConverters::getRollupIndexCaps, options, GetRollupIndexCapsResponse::fromXContent, diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java index a807b798f57b5..a25396af648dd 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java @@ -113,9 +113,10 @@ public GetUsersResponse getUsers(GetUsersRequest request, RequestOptions options * @param request the request with the user's name * @param options the request options (e.g., headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getUsersAsync(GetUsersRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getUsers, options, + public Cancellable getUsersAsync(GetUsersRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getUsers, options, GetUsersResponse::fromXContent, listener, emptySet()); } @@ -142,9 +143,10 @@ public PutUserResponse putUser(PutUserRequest request, RequestOptions options) t * @param request the request with the user's information * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putUserAsync(PutUserRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putUser, options, + public Cancellable putUserAsync(PutUserRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putUser, options, PutUserResponse::fromXContent, listener, emptySet()); } @@ -169,9 +171,10 @@ public DeleteUserResponse deleteUser(DeleteUserRequest request, RequestOptions o * @param request the request with the user to delete * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteUserAsync(DeleteUserRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::deleteUser, options, + public Cancellable deleteUserAsync(DeleteUserRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::deleteUser, options, DeleteUserResponse::fromXContent, listener, singleton(404)); } @@ -196,10 +199,11 @@ public PutRoleMappingResponse putRoleMapping(final PutRoleMappingRequest request * @param request the request with the role mapping information * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putRoleMappingAsync(final PutRoleMappingRequest request, final RequestOptions options, - final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putRoleMapping, options, + public Cancellable putRoleMappingAsync(final PutRoleMappingRequest request, final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putRoleMapping, options, PutRoleMappingResponse::fromXContent, listener, emptySet()); } @@ -216,7 +220,8 @@ public void putRoleMappingAsync(final PutRoleMappingRequest request, final Reque * @throws IOException in case there is a problem sending the request or * parsing back the response */ - public GetRoleMappingsResponse getRoleMappings(final GetRoleMappingsRequest request, final RequestOptions options) throws IOException { + public GetRoleMappingsResponse getRoleMappings(final GetRoleMappingsRequest request, + final RequestOptions options) throws IOException { return restHighLevelClient.performRequestAndParseEntity(request, SecurityRequestConverters::getRoleMappings, options, GetRoleMappingsResponse::fromXContent, emptySet()); } @@ -230,10 +235,11 @@ public GetRoleMappingsResponse getRoleMappings(final GetRoleMappingsRequest requ * If no role mapping name is provided then retrieves all role mappings. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getRoleMappingsAsync(final GetRoleMappingsRequest request, final RequestOptions options, - final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getRoleMappings, + public Cancellable getRoleMappingsAsync(final GetRoleMappingsRequest request, final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getRoleMappings, options, GetRoleMappingsResponse::fromXContent, listener, emptySet()); } @@ -276,10 +282,11 @@ public boolean enableUser(RequestOptions options, EnableUserRequest request) thr * @param request the request with the user to enable * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void enableUserAsync(EnableUserRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsync(request, SecurityRequestConverters::enableUser, options, + public Cancellable enableUserAsync(EnableUserRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsync(request, SecurityRequestConverters::enableUser, options, RestHighLevelClient::convertExistsResponse, listener, emptySet()); } @@ -292,11 +299,12 @@ public void enableUserAsync(EnableUserRequest request, RequestOptions options, * @param request the request with the user to enable * @param listener the listener to be notified upon request completion * @deprecated use {@link #enableUserAsync(EnableUserRequest, RequestOptions, ActionListener)} instead + * @return cancellable that may be used to cancel the request */ @Deprecated - public void enableUserAsync(RequestOptions options, EnableUserRequest request, - ActionListener listener) { - enableUserAsync(request, options, listener); + public Cancellable enableUserAsync(RequestOptions options, EnableUserRequest request, + ActionListener listener) { + return enableUserAsync(request, options, listener); } /** @@ -338,10 +346,11 @@ public boolean disableUser(RequestOptions options, DisableUserRequest request) t * @param request the request with the user to disable * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void disableUserAsync(DisableUserRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsync(request, SecurityRequestConverters::disableUser, options, + public Cancellable disableUserAsync(DisableUserRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsync(request, SecurityRequestConverters::disableUser, options, RestHighLevelClient::convertExistsResponse, listener, emptySet()); } @@ -354,11 +363,12 @@ public void disableUserAsync(DisableUserRequest request, RequestOptions options, * @param request the request with the user to disable * @param listener the listener to be notified upon request completion * @deprecated use {@link #disableUserAsync(DisableUserRequest, RequestOptions, ActionListener)} instead + * @return cancellable that may be used to cancel the request */ @Deprecated - public void disableUserAsync(RequestOptions options, DisableUserRequest request, - ActionListener listener) { - disableUserAsync(request, options, listener); + public Cancellable disableUserAsync(RequestOptions options, DisableUserRequest request, + ActionListener listener) { + return disableUserAsync(request, options, listener); } /** @@ -381,9 +391,10 @@ public AuthenticateResponse authenticate(RequestOptions options) throws IOExcept * * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void authenticateAsync(RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(AuthenticateRequest.INSTANCE, AuthenticateRequest::getRequest, options, + public Cancellable authenticateAsync(RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(AuthenticateRequest.INSTANCE, AuthenticateRequest::getRequest, options, AuthenticateResponse::fromXContent, listener, emptySet()); } @@ -405,13 +416,14 @@ public HasPrivilegesResponse hasPrivileges(HasPrivilegesRequest request, Request * Asynchronously determine whether the current user has a specified list of privileges * See * the docs for more. - * * @param request the request with the privileges to check * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void hasPrivilegesAsync(HasPrivilegesRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::hasPrivileges, options, + public Cancellable hasPrivilegesAsync(HasPrivilegesRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::hasPrivileges, options, HasPrivilegesResponse::fromXContent, listener, emptySet()); } @@ -428,9 +440,11 @@ public GetUserPrivilegesResponse getUserPrivileges(RequestOptions options) throw * Asynchronously retrieve the set of effective privileges held by the current user. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getUserPrivilegesAsync(RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(GetUserPrivilegesRequest.INSTANCE, GetUserPrivilegesRequest::getRequest, + public Cancellable getUserPrivilegesAsync(RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + GetUserPrivilegesRequest.INSTANCE, GetUserPrivilegesRequest::getRequest, options, GetUserPrivilegesResponse::fromXContent, listener, emptySet()); } @@ -457,10 +471,11 @@ public ClearRealmCacheResponse clearRealmCache(ClearRealmCacheRequest request, R * @param request the request with the realm names and usernames to clear the cache for * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void clearRealmCacheAsync(ClearRealmCacheRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::clearRealmCache, options, + public Cancellable clearRealmCacheAsync(ClearRealmCacheRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::clearRealmCache, options, ClearRealmCacheResponse::fromXContent, listener, emptySet()); } @@ -487,10 +502,11 @@ public ClearRolesCacheResponse clearRolesCache(ClearRolesCacheRequest request, R * @param request the request with the roles for which the cache should be cleared. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void clearRolesCacheAsync(ClearRolesCacheRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::clearRolesCache, options, + public Cancellable clearRolesCacheAsync(ClearRolesCacheRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::clearRolesCache, options, ClearRolesCacheResponse::fromXContent, listener, emptySet()); } @@ -515,9 +531,11 @@ public GetSslCertificatesResponse getSslCertificates(RequestOptions options) thr * * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getSslCertificatesAsync(RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(GetSslCertificatesRequest.INSTANCE, GetSslCertificatesRequest::getRequest, + public Cancellable getSslCertificatesAsync(RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( + GetSslCertificatesRequest.INSTANCE, GetSslCertificatesRequest::getRequest, options, GetSslCertificatesResponse::fromXContent, listener, emptySet()); } @@ -560,10 +578,11 @@ public boolean changePassword(RequestOptions options, ChangePasswordRequest requ * @param request the request with the user's new password * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void changePasswordAsync(ChangePasswordRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsync(request, SecurityRequestConverters::changePassword, options, + public Cancellable changePasswordAsync(ChangePasswordRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsync(request, SecurityRequestConverters::changePassword, options, RestHighLevelClient::convertExistsResponse, listener, emptySet()); } @@ -576,11 +595,12 @@ public void changePasswordAsync(ChangePasswordRequest request, RequestOptions op * @param request the request with the user's new password * @param listener the listener to be notified upon request completion * @deprecated use {@link #changePasswordAsync(ChangePasswordRequest, RequestOptions, ActionListener)} instead + * @return cancellable that may be used to cancel the request */ @Deprecated - public void changePasswordAsync(RequestOptions options, ChangePasswordRequest request, - ActionListener listener) { - changePasswordAsync(request, options, listener); + public Cancellable changePasswordAsync(RequestOptions options, ChangePasswordRequest request, + ActionListener listener) { + return changePasswordAsync(request, options, listener); } /** @@ -605,9 +625,10 @@ public DeleteRoleMappingResponse deleteRoleMapping(DeleteRoleMappingRequest requ * @param request the request with the roles to get * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getRolesAsync(GetRolesRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getRoles, options, + public Cancellable getRolesAsync(GetRolesRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getRoles, options, GetRolesResponse::fromXContent, listener, emptySet()); } @@ -634,9 +655,10 @@ public GetRolesResponse getRoles(final GetRolesRequest request, final RequestOpt * @param request the request containing the role to create or update * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putRoleAsync(PutRoleRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putRole, options, + public Cancellable putRoleAsync(PutRoleRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putRole, options, PutRoleResponse::fromXContent, listener, emptySet()); } @@ -662,10 +684,12 @@ public PutRoleResponse putRole(final PutRoleRequest request, final RequestOption * @param request the request with the role mapping name to be deleted. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteRoleMappingAsync(DeleteRoleMappingRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::deleteRoleMapping, options, + public Cancellable deleteRoleMappingAsync(DeleteRoleMappingRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, + SecurityRequestConverters::deleteRoleMapping, options, DeleteRoleMappingResponse::fromXContent, listener, emptySet()); } @@ -690,9 +714,11 @@ public DeleteRoleResponse deleteRole(DeleteRoleRequest request, RequestOptions o * @param request the request with the role to delete * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteRoleAsync(DeleteRoleRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::deleteRole, options, + public Cancellable deleteRoleAsync(DeleteRoleRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::deleteRole, options, DeleteRoleResponse::fromXContent, listener, singleton(404)); } @@ -719,9 +745,11 @@ public CreateTokenResponse createToken(CreateTokenRequest request, RequestOption * @param request the request for the token * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void createTokenAsync(CreateTokenRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::createToken, options, + public Cancellable createTokenAsync(CreateTokenRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::createToken, options, CreateTokenResponse::fromXContent, listener, emptySet()); } @@ -744,14 +772,14 @@ public InvalidateTokenResponse invalidateToken(InvalidateTokenRequest request, R * Asynchronously invalidates an OAuth2 token. * See * the docs for more. - * * @param request the request to invalidate the token * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void invalidateTokenAsync(InvalidateTokenRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::invalidateToken, options, + public Cancellable invalidateTokenAsync(InvalidateTokenRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::invalidateToken, options, InvalidateTokenResponse::fromXContent, listener, emptySet()); } @@ -777,10 +805,13 @@ public GetBuiltinPrivilegesResponse getBuiltinPrivileges(final RequestOptions op * * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getBuiltinPrivilegesAsync(final RequestOptions options, final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(GetBuiltinPrivilegesRequest.INSTANCE, - GetBuiltinPrivilegesRequest::getRequest, options, GetBuiltinPrivilegesResponse::fromXContent, listener, emptySet()); + public Cancellable getBuiltinPrivilegesAsync(final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(GetBuiltinPrivilegesRequest.INSTANCE, + GetBuiltinPrivilegesRequest::getRequest, options, GetBuiltinPrivilegesResponse::fromXContent, + listener, emptySet()); } /** @@ -806,16 +837,16 @@ public GetPrivilegesResponse getPrivileges(final GetPrivilegesRequest request, f * Asynchronously get application privilege(s). * See * the docs for more. - * - * @param request {@link GetPrivilegesRequest} with the application name and the privilege name. + * @param request {@link GetPrivilegesRequest} with the application name and the privilege name. * If no application name is provided, information about all privileges for all applications is retrieved. * If no privilege name is provided, information about all privileges of the specified application is retrieved. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getPrivilegesAsync(final GetPrivilegesRequest request, final RequestOptions options, - final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getPrivileges, + public Cancellable getPrivilegesAsync(final GetPrivilegesRequest request, final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getPrivileges, options, GetPrivilegesResponse::fromXContent, listener, emptySet()); } @@ -844,10 +875,11 @@ public PutPrivilegesResponse putPrivileges(final PutPrivilegesRequest request, f * @param options the request options (e.g. headers), use * {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putPrivilegesAsync(final PutPrivilegesRequest request, final RequestOptions options, - final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putPrivileges, options, + public Cancellable putPrivilegesAsync(final PutPrivilegesRequest request, final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::putPrivileges, options, PutPrivilegesResponse::fromXContent, listener, emptySet()); } @@ -874,10 +906,11 @@ public DeletePrivilegesResponse deletePrivileges(DeletePrivilegesRequest request * @param request the request with the application privilege to delete * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deletePrivilegesAsync(DeletePrivilegesRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::deletePrivileges, options, + public Cancellable deletePrivilegesAsync(DeletePrivilegesRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::deletePrivileges, options, DeletePrivilegesResponse::fromXContent, listener, singleton(404)); } @@ -904,10 +937,11 @@ public CreateApiKeyResponse createApiKey(final CreateApiKeyRequest request, fina * @param request the request to create a API key * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void createApiKeyAsync(final CreateApiKeyRequest request, final RequestOptions options, - final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::createApiKey, options, + public Cancellable createApiKeyAsync(final CreateApiKeyRequest request, final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::createApiKey, options, CreateApiKeyResponse::fromXContent, listener, emptySet()); } @@ -934,10 +968,11 @@ public GetApiKeyResponse getApiKey(final GetApiKeyRequest request, final Request * @param request the request to retrieve API key(s) * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getApiKeyAsync(final GetApiKeyRequest request, final RequestOptions options, - final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getApiKey, options, + public Cancellable getApiKeyAsync(final GetApiKeyRequest request, final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::getApiKey, options, GetApiKeyResponse::fromXContent, listener, emptySet()); } @@ -965,10 +1000,11 @@ public InvalidateApiKeyResponse invalidateApiKey(final InvalidateApiKeyRequest r * @param request the request to invalidate API key(s) * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void invalidateApiKeyAsync(final InvalidateApiKeyRequest request, final RequestOptions options, - final ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::invalidateApiKey, options, + public Cancellable invalidateApiKeyAsync(final InvalidateApiKeyRequest request, final RequestOptions options, + final ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::invalidateApiKey, options, InvalidateApiKeyResponse::fromXContent, listener, emptySet()); } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotClient.java index d3b2ea466f458..134dc921c450d 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/SnapshotClient.java @@ -79,10 +79,12 @@ public GetRepositoriesResponse getRepository(GetRepositoriesRequest getRepositor * @param getRepositoriesRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getRepositoryAsync(GetRepositoriesRequest getRepositoriesRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getRepositoriesRequest, SnapshotRequestConverters::getRepositories, options, + public Cancellable getRepositoryAsync(GetRepositoriesRequest getRepositoriesRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getRepositoriesRequest, + SnapshotRequestConverters::getRepositories, options, GetRepositoriesResponse::fromXContent, listener, emptySet()); } @@ -107,10 +109,12 @@ public AcknowledgedResponse createRepository(PutRepositoryRequest putRepositoryR * @param putRepositoryRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void createRepositoryAsync(PutRepositoryRequest putRepositoryRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(putRepositoryRequest, SnapshotRequestConverters::createRepository, options, + public Cancellable createRepositoryAsync(PutRepositoryRequest putRepositoryRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(putRepositoryRequest, + SnapshotRequestConverters::createRepository, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -136,10 +140,12 @@ public AcknowledgedResponse deleteRepository(DeleteRepositoryRequest deleteRepos * @param deleteRepositoryRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteRepositoryAsync(DeleteRepositoryRequest deleteRepositoryRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(deleteRepositoryRequest, SnapshotRequestConverters::deleteRepository, options, + public Cancellable deleteRepositoryAsync(DeleteRepositoryRequest deleteRepositoryRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(deleteRepositoryRequest, + SnapshotRequestConverters::deleteRepository, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -165,10 +171,12 @@ public VerifyRepositoryResponse verifyRepository(VerifyRepositoryRequest verifyR * @param verifyRepositoryRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void verifyRepositoryAsync(VerifyRepositoryRequest verifyRepositoryRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(verifyRepositoryRequest, SnapshotRequestConverters::verifyRepository, options, + public Cancellable verifyRepositoryAsync(VerifyRepositoryRequest verifyRepositoryRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(verifyRepositoryRequest, + SnapshotRequestConverters::verifyRepository, options, VerifyRepositoryResponse::fromXContent, listener, emptySet()); } @@ -194,10 +202,11 @@ public CleanupRepositoryResponse cleanupRepository(CleanupRepositoryRequest clea * @param cleanupRepositoryRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void cleanupRepositoryAsync(CleanupRepositoryRequest cleanupRepositoryRequest, RequestOptions options, + public Cancellable cleanupRepositoryAsync(CleanupRepositoryRequest cleanupRepositoryRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(cleanupRepositoryRequest, SnapshotRequestConverters::cleanupRepository, + return restHighLevelClient.performRequestAsyncAndParseEntity(cleanupRepositoryRequest, SnapshotRequestConverters::cleanupRepository, options, CleanupRepositoryResponse::fromXContent, listener, emptySet()); } @@ -218,10 +227,12 @@ public CreateSnapshotResponse create(CreateSnapshotRequest createSnapshotRequest *

* See Snapshot and Restore * API on elastic.co + * @return cancellable that may be used to cancel the request */ - public void createAsync(CreateSnapshotRequest createSnapshotRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(createSnapshotRequest, SnapshotRequestConverters::createSnapshot, options, + public Cancellable createAsync(CreateSnapshotRequest createSnapshotRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(createSnapshotRequest, + SnapshotRequestConverters::createSnapshot, options, CreateSnapshotResponse::fromXContent, listener, emptySet()); } @@ -244,13 +255,15 @@ public GetSnapshotsResponse get(GetSnapshotsRequest getSnapshotsRequest, Request * Asynchronously get snapshots. * See Snapshot and Restore * API on elastic.co - * - * @param getSnapshotsRequest the request + * @param getSnapshotsRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getAsync(GetSnapshotsRequest getSnapshotsRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(getSnapshotsRequest, SnapshotRequestConverters::getSnapshots, options, + public Cancellable getAsync(GetSnapshotsRequest getSnapshotsRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(getSnapshotsRequest, + SnapshotRequestConverters::getSnapshots, options, GetSnapshotsResponse::fromXContent, listener, emptySet()); } @@ -276,10 +289,12 @@ public SnapshotsStatusResponse status(SnapshotsStatusRequest snapshotsStatusRequ * @param snapshotsStatusRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void statusAsync(SnapshotsStatusRequest snapshotsStatusRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(snapshotsStatusRequest, SnapshotRequestConverters::snapshotsStatus, options, + public Cancellable statusAsync(SnapshotsStatusRequest snapshotsStatusRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(snapshotsStatusRequest, + SnapshotRequestConverters::snapshotsStatus, options, SnapshotsStatusResponse::fromXContent, listener, emptySet()); } @@ -306,10 +321,12 @@ public RestoreSnapshotResponse restore(RestoreSnapshotRequest restoreSnapshotReq * @param restoreSnapshotRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void restoreAsync(RestoreSnapshotRequest restoreSnapshotRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(restoreSnapshotRequest, SnapshotRequestConverters::restoreSnapshot, options, + public Cancellable restoreAsync(RestoreSnapshotRequest restoreSnapshotRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(restoreSnapshotRequest, + SnapshotRequestConverters::restoreSnapshot, options, RestoreSnapshotResponse::fromXContent, listener, emptySet()); } @@ -324,7 +341,8 @@ public void restoreAsync(RestoreSnapshotRequest restoreSnapshotRequest, RequestO * @throws IOException in case there is a problem sending the request or parsing back the response */ public AcknowledgedResponse delete(DeleteSnapshotRequest deleteSnapshotRequest, RequestOptions options) throws IOException { - return restHighLevelClient.performRequestAndParseEntity(deleteSnapshotRequest, SnapshotRequestConverters::deleteSnapshot, options, + return restHighLevelClient.performRequestAndParseEntity(deleteSnapshotRequest, + SnapshotRequestConverters::deleteSnapshot, options, AcknowledgedResponse::fromXContent, emptySet()); } @@ -336,10 +354,12 @@ public AcknowledgedResponse delete(DeleteSnapshotRequest deleteSnapshotRequest, * @param deleteSnapshotRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteAsync(DeleteSnapshotRequest deleteSnapshotRequest, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(deleteSnapshotRequest, SnapshotRequestConverters::deleteSnapshot, options, + public Cancellable deleteAsync(DeleteSnapshotRequest deleteSnapshotRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(deleteSnapshotRequest, + SnapshotRequestConverters::deleteSnapshot, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksClient.java index 4bf7565222a73..04ccd3239335f 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/TasksClient.java @@ -65,12 +65,13 @@ public ListTasksResponse list(ListTasksRequest request, RequestOptions options) * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void listAsync(ListTasksRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, TasksRequestConverters::listTasks, options, + public Cancellable listAsync(ListTasksRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, TasksRequestConverters::listTasks, options, ListTasksResponse::fromXContent, listener, emptySet()); } - + /** * Get a task using the Task Management API. * See @@ -82,9 +83,9 @@ public void listAsync(ListTasksRequest request, RequestOptions options, ActionLi */ public Optional get(GetTaskRequest request, RequestOptions options) throws IOException { return restHighLevelClient.performRequestAndParseOptionalEntity(request, TasksRequestConverters::getTask, options, - GetTaskResponse::fromXContent); - } - + GetTaskResponse::fromXContent); + } + /** * Get a task using the Task Management API. * See @@ -92,12 +93,14 @@ public Optional get(GetTaskRequest request, RequestOptions opti * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener an actionlistener that takes an optional response (404s are returned as an empty Optional) + * @return cancellable that may be used to cancel the request */ - public void getAsync(GetTaskRequest request, RequestOptions options, ActionListener> listener) { - - restHighLevelClient.performRequestAsyncAndParseOptionalEntity(request, TasksRequestConverters::getTask, options, + public Cancellable getAsync(GetTaskRequest request, RequestOptions options, + ActionListener> listener) { + + return restHighLevelClient.performRequestAsyncAndParseOptionalEntity(request, TasksRequestConverters::getTask, options, GetTaskResponse::fromXContent, listener); - } + } /** * Cancel one or more cluster tasks using the Task Management API. @@ -128,9 +131,11 @@ public CancelTasksResponse cancel(CancelTasksRequest cancelTasksRequest, Request * @param cancelTasksRequest the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void cancelAsync(CancelTasksRequest cancelTasksRequest, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable cancelAsync(CancelTasksRequest cancelTasksRequest, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( cancelTasksRequest, TasksRequestConverters::cancelTasks, options, diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherClient.java index 9d75132a903c3..9a1ad6815aa99 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherClient.java @@ -71,10 +71,11 @@ public AcknowledgedResponse startWatchService(StartWatchServiceRequest request, * See * the docs for more. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return cancellable that may be used to cancel the request */ - public void startWatchServiceAsync(StartWatchServiceRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable startWatchServiceAsync(StartWatchServiceRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, WatcherRequestConverters::startWatchService, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -98,10 +99,11 @@ public AcknowledgedResponse stopWatchService(StopWatchServiceRequest request, Re * the docs for more. * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized + * @return cancellable that may be used to cancel the request */ - public void stopWatchServiceAsync(StopWatchServiceRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity( + public Cancellable stopWatchServiceAsync(StopWatchServiceRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity( request, WatcherRequestConverters::stopWatchService, options, AcknowledgedResponse::fromXContent, listener, emptySet()); } @@ -126,10 +128,11 @@ public PutWatchResponse putWatch(PutWatchRequest request, RequestOptions options * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void putWatchAsync(PutWatchRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::putWatch, options, + public Cancellable putWatchAsync(PutWatchRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::putWatch, options, PutWatchResponse::fromXContent, listener, emptySet()); } @@ -154,10 +157,11 @@ public GetWatchResponse getWatch(GetWatchRequest request, RequestOptions options * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void getWatchAsync(GetWatchRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::getWatch, options, + public Cancellable getWatchAsync(GetWatchRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::getWatch, options, GetWatchResponse::fromXContent, listener, emptySet()); } @@ -183,10 +187,11 @@ public DeactivateWatchResponse deactivateWatch(DeactivateWatchRequest request, R * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deactivateWatchAsync(DeactivateWatchRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::deactivateWatch, options, + public Cancellable deactivateWatchAsync(DeactivateWatchRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::deactivateWatch, options, DeactivateWatchResponse::fromXContent, listener, emptySet()); } @@ -211,9 +216,10 @@ public DeleteWatchResponse deleteWatch(DeleteWatchRequest request, RequestOption * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void deleteWatchAsync(DeleteWatchRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::deleteWatch, options, + public Cancellable deleteWatchAsync(DeleteWatchRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::deleteWatch, options, DeleteWatchResponse::fromXContent, listener, singleton(404)); } @@ -238,9 +244,10 @@ public AckWatchResponse ackWatch(AckWatchRequest request, RequestOptions options * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon completion of the request + * @return cancellable that may be used to cancel the request */ - public void ackWatchAsync(AckWatchRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::ackWatch, options, + public Cancellable ackWatchAsync(AckWatchRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::ackWatch, options, AckWatchResponse::fromXContent, listener, emptySet()); } @@ -265,9 +272,11 @@ public ActivateWatchResponse activateWatch(ActivateWatchRequest request, Request * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void activateWatchAsync(ActivateWatchRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::activateWatch, options, + public Cancellable activateWatchAsync(ActivateWatchRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::activateWatch, options, ActivateWatchResponse::fromXContent, listener, singleton(404)); } @@ -292,9 +301,11 @@ public ExecuteWatchResponse executeWatch(ExecuteWatchRequest request, RequestOpt * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notifed upon request completion + * @return cancellable that may be used to cancel the request */ - public void executeWatchAsync(ExecuteWatchRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::executeWatch, options, + public Cancellable executeWatchAsync(ExecuteWatchRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::executeWatch, options, ExecuteWatchResponse::fromXContent, listener, emptySet()); } @@ -319,9 +330,11 @@ public WatcherStatsResponse watcherStats(WatcherStatsRequest request, RequestOpt * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void watcherStatsAsync(WatcherStatsRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::watcherStats, options, + public Cancellable watcherStatsAsync(WatcherStatsRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, WatcherRequestConverters::watcherStats, options, WatcherStatsResponse::fromXContent, listener, emptySet()); } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java index 69cdd329e3984..cc020de92f6a8 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/XPackClient.java @@ -67,10 +67,11 @@ public XPackInfoResponse info(XPackInfoRequest request, RequestOptions options) * @param request the request * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void infoAsync(XPackInfoRequest request, RequestOptions options, - ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, XPackRequestConverters::info, options, + public Cancellable infoAsync(XPackInfoRequest request, RequestOptions options, + ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, XPackRequestConverters::info, options, XPackInfoResponse::fromXContent, listener, emptySet()); } @@ -89,9 +90,10 @@ public XPackUsageResponse usage(XPackUsageRequest request, RequestOptions option * Asynchronously fetch usage information about X-Pack features from the cluster. * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion + * @return cancellable that may be used to cancel the request */ - public void usageAsync(XPackUsageRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, XPackRequestConverters::usage, options, + public Cancellable usageAsync(XPackUsageRequest request, RequestOptions options, ActionListener listener) { + return restHighLevelClient.performRequestAsyncAndParseEntity(request, XPackRequestConverters::usage, options, XPackUsageResponse::fromXContent, listener, emptySet()); } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java index e89035598bc5d..788716ecce7dc 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java @@ -900,7 +900,7 @@ private static void assertSyncMethod(Method method, String apiName, List private static void assertAsyncMethod(Map> methods, Method method, String apiName) { assertTrue("async method [" + method.getName() + "] doesn't have corresponding sync method", methods.containsKey(apiName.substring(0, apiName.length() - 6))); - assertThat("async method [" + method + "] should return void", method.getReturnType(), equalTo(Void.TYPE)); + assertThat("async method [" + method + "] should return Cancellable", method.getReturnType(), equalTo(Cancellable.class)); assertEquals("async method [" + method + "] should not throw any exceptions", 0, method.getExceptionTypes().length); if (APIS_WITHOUT_REQUEST_OBJECT.contains(apiName.replaceAll("_async$", ""))) { assertEquals(2, method.getParameterTypes().length); From ccfa5a2c435ad4be14ceb693cb1d6c7a4e2189a1 Mon Sep 17 00:00:00 2001 From: Jilles van Gurp Date: Wed, 28 Aug 2019 10:59:33 +0200 Subject: [PATCH 3/5] Add a few notes on Cancellable to the LLRC and HLRC docs. (#45912) Add a section to both the low level and high level client documentation on asynchronous usage and `Cancellable` added for #44802 Co-Authored-By: Lee Hinman --- .../high-level/getting-started.asciidoc | 20 +++++++++++++++++++ docs/java-rest/low-level/usage.asciidoc | 6 ++++++ 2 files changed, 26 insertions(+) diff --git a/docs/java-rest/high-level/getting-started.asciidoc b/docs/java-rest/high-level/getting-started.asciidoc index 89912cc2a4593..0f3b66e667401 100644 --- a/docs/java-rest/high-level/getting-started.asciidoc +++ b/docs/java-rest/high-level/getting-started.asciidoc @@ -154,3 +154,23 @@ executes the request. For example, this is the place where you'd specify a `NodeSelector` to control which node receives the request. See the <> for more examples of customizing the options. +=== Asynchronous usage + +All of the the methods across the different clients exist in a traditional synchronous and +asynchronous variant. The difference is that the asynchronous ones use asynchronous requests +in the REST Low Level Client. This is useful if you are doing multiple requests or are using e.g. +rx java, Kotlin co-routines, or similar frameworks. + +The asynchronous methods are recognizable by the fact that they have the word "Async" in their name +and return a `Cancellable` instance. The asynchronous methods accept the same request object +as the synchronous variant and accept a generic `ActionListener` where `T` is the return +type of the synchronous method. + +All asynchronous methods return a `Cancellable` object with a `cancel` method that you may call +in case you want to abort the request. Cancelling +no longer needed requests is a good way to avoid putting unnecessary +load on Elasticsearch. + +Using the `Cancellable` instance is optional and you can safely ignore this if you have +no need for this. A use case for this would be using this with e.g. Kotlin's `suspendCancellableCoRoutine`. + diff --git a/docs/java-rest/low-level/usage.asciidoc b/docs/java-rest/low-level/usage.asciidoc index d0f4b070a55d6..9d55ff79ce261 100644 --- a/docs/java-rest/low-level/usage.asciidoc +++ b/docs/java-rest/low-level/usage.asciidoc @@ -338,6 +338,12 @@ the underlying http client. On the server side, this does not automatically translate to the execution of that request being cancelled, which needs to be specifically implemented in the API itself. +The use of the `Cancellable` instance is optional and you can safely ignore this +if you don't need it. A typical usecase for this would be using this together with +frameworks like Rx Java or the Kotlin's `suspendCancellableCoRoutine`. Cancelling +no longer needed requests is a good way to avoid putting unnecessary +load on Elasticsearch. + ["source","java",subs="attributes,callouts,macros"] -------------------------------------------------- include-tagged::{doc-tests}/RestClientDocumentation.java[rest-client-async-cancel] From a1fbc0a1178ca2c096c3198df953db12dfc7a715 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 11 Sep 2019 13:53:57 +0200 Subject: [PATCH 4/5] Add SearchRestCancellationIT This test verifies automatic cancellation of search requests on connection close. It was previously not present in 7.x as the http client was subject do a bug which made testing cancellation of requests impossible. Now that the bug is fixed upstream, we can also backport this test --- .../http/SearchRestCancellationIT.java | 251 ++++++++++++++++++ 1 file changed, 251 insertions(+) create mode 100644 qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java new file mode 100644 index 0000000000000..698f42c43ca03 --- /dev/null +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java @@ -0,0 +1,251 @@ +/* + * 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. + */ +package org.elasticsearch.http; + +import org.apache.logging.log4j.LogManager; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; +import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; +import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.Cancellable; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseListener; +import org.elasticsearch.common.Strings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.PluginsService; +import org.elasticsearch.script.MockScriptPlugin; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptType; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.lookup.LeafFieldsLookup; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.transport.TransportService; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; + +import static org.elasticsearch.index.query.QueryBuilders.scriptQuery; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.instanceOf; + +public class SearchRestCancellationIT extends HttpSmokeTestCase { + + @Override + protected Collection> nodePlugins() { + List> plugins = new ArrayList<>(); + plugins.add(ScriptedBlockPlugin.class); + plugins.addAll(super.nodePlugins()); + return plugins; + } + + public void testAutomaticCancellationDuringQueryPhase() throws Exception { + Map nodeIdToName = readNodesInfo(); + + List plugins = initBlockFactory(); + indexTestData(); + + Request searchRequest = new Request("GET", "/test/_search"); + SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( + new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap()))); + searchRequest.setJsonEntity(Strings.toString(searchSource)); + + CountDownLatch latch = new CountDownLatch(1); + AtomicReference error = new AtomicReference<>(); + Cancellable cancellable = getRestClient().performRequestAsync(searchRequest, new ResponseListener() { + @Override + public void onSuccess(Response response) { + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + error.set(exception); + latch.countDown(); + } + }); + + awaitForBlock(plugins); + cancellable.cancel(); + ensureSearchTaskIsCancelled(nodeIdToName::get); + + disableBlocks(plugins); + latch.await(); + assertThat(error.get(), instanceOf(CancellationException.class)); + } + + public void testAutomaticCancellationDuringFetchPhase() throws Exception { + Map nodeIdToName = readNodesInfo(); + + List plugins = initBlockFactory(); + indexTestData(); + + Request searchRequest = new Request("GET", "/test/_search"); + SearchSourceBuilder searchSource = new SearchSourceBuilder().scriptField("test_field", + new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap())); + searchRequest.setJsonEntity(Strings.toString(searchSource)); + + CountDownLatch latch = new CountDownLatch(1); + AtomicReference error = new AtomicReference<>(); + Cancellable cancellable = getRestClient().performRequestAsync(searchRequest, new ResponseListener() { + @Override + public void onSuccess(Response response) { + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + error.set(exception); + latch.countDown(); + } + }); + + awaitForBlock(plugins); + cancellable.cancel(); + ensureSearchTaskIsCancelled(nodeIdToName::get); + + disableBlocks(plugins); + latch.await(); + assertThat(error.get(), instanceOf(CancellationException.class)); + } + + private static Map readNodesInfo() { + Map nodeIdToName = new HashMap<>(); + NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); + assertFalse(nodesInfoResponse.hasFailures()); + for (NodeInfo node : nodesInfoResponse.getNodes()) { + nodeIdToName.put(node.getNode().getId(), node.getNode().getName()); + } + return nodeIdToName; + } + + private static void ensureSearchTaskIsCancelled(Function nodeIdToName) throws Exception { + SetOnce searchTask = new SetOnce<>(); + ListTasksResponse listTasksResponse = client().admin().cluster().prepareListTasks().get(); + for (TaskInfo task : listTasksResponse.getTasks()) { + if (task.getAction().equals(SearchAction.NAME)) { + searchTask.set(task); + } + } + assertNotNull(searchTask.get()); + TaskId taskId = searchTask.get().getTaskId(); + String nodeName = nodeIdToName.apply(taskId.getNodeId()); + assertBusy(() -> { + TaskManager taskManager = internalCluster().getInstance(TransportService.class, nodeName).getTaskManager(); + Task task = taskManager.getTask(taskId.getId()); + assertThat(task, instanceOf(CancellableTask.class)); + assertTrue(((CancellableTask)task).isCancelled()); + }); + } + + private static void indexTestData() { + for (int i = 0; i < 5; i++) { + // Make sure we have a few segments + BulkRequestBuilder bulkRequestBuilder = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + for (int j = 0; j < 20; j++) { + bulkRequestBuilder.add(client().prepareIndex("test", "_doc", Integer.toString(i * 5 + j)).setSource("field", "value")); + } + assertNoFailures(bulkRequestBuilder.get()); + } + } + + private static List initBlockFactory() { + List plugins = new ArrayList<>(); + for (PluginsService pluginsService : internalCluster().getDataNodeInstances(PluginsService.class)) { + plugins.addAll(pluginsService.filterPlugins(ScriptedBlockPlugin.class)); + } + for (ScriptedBlockPlugin plugin : plugins) { + plugin.reset(); + plugin.enableBlock(); + } + return plugins; + } + + private void awaitForBlock(List plugins) throws Exception { + int numberOfShards = getNumShards("test").numPrimaries; + assertBusy(() -> { + int numberOfBlockedPlugins = 0; + for (ScriptedBlockPlugin plugin : plugins) { + numberOfBlockedPlugins += plugin.hits.get(); + } + logger.info("The plugin blocked on {} out of {} shards", numberOfBlockedPlugins, numberOfShards); + assertThat(numberOfBlockedPlugins, greaterThan(0)); + }, 10, TimeUnit.SECONDS); + } + + private static void disableBlocks(List plugins) { + for (ScriptedBlockPlugin plugin : plugins) { + plugin.disableBlock(); + } + } + + public static class ScriptedBlockPlugin extends MockScriptPlugin { + static final String SCRIPT_NAME = "search_block"; + + private final AtomicInteger hits = new AtomicInteger(); + + private final AtomicBoolean shouldBlock = new AtomicBoolean(true); + + void reset() { + hits.set(0); + } + + void disableBlock() { + shouldBlock.set(false); + } + + void enableBlock() { + shouldBlock.set(true); + } + + @Override + public Map, Object>> pluginScripts() { + return Collections.singletonMap(SCRIPT_NAME, params -> { + LeafFieldsLookup fieldsLookup = (LeafFieldsLookup) params.get("_fields"); + LogManager.getLogger(SearchRestCancellationIT.class).info("Blocking on the document {}", fieldsLookup.get("_id")); + hits.incrementAndGet(); + try { + awaitBusy(() -> shouldBlock.get() == false); + } catch (Exception e) { + throw new RuntimeException(e); + } + return true; + }); + } + } +} From 2858816ad2c44065561ea8f772b9ed563490a038 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 11 Sep 2019 13:42:32 +0200 Subject: [PATCH 5/5] Update http-core and http-client dependencies (#46549) Relates to #45808 Closes #45577 --- buildSrc/version.properties | 4 +-- .../client/IndexLifecycleClient.java | 4 +-- .../elasticsearch/client/SecurityClient.java | 8 +++--- .../rest/licenses/httpclient-4.5.10.jar.sha1 | 1 + .../rest/licenses/httpclient-4.5.8.jar.sha1 | 1 - client/rest/licenses/httpcore-4.4.11.jar.sha1 | 1 - client/rest/licenses/httpcore-4.4.12.jar.sha1 | 1 + .../licenses/httpcore-nio-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-nio-4.4.12.jar.sha1 | 1 + .../RestClientMultipleHostsIntegTests.java | 25 +++++++++++++------ .../RestClientSingleHostIntegTests.java | 15 ++++++++--- .../licenses/httpclient-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-4.5.8.jar.sha1 | 1 - .../sniffer/licenses/httpcore-4.4.11.jar.sha1 | 1 - .../sniffer/licenses/httpcore-4.4.12.jar.sha1 | 1 + .../licenses/httpclient-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-4.5.8.jar.sha1 | 1 - .../licenses/httpcore-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-4.4.12.jar.sha1 | 1 + .../licenses/httpclient-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-4.5.8.jar.sha1 | 1 - .../licenses/httpcore-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-4.4.12.jar.sha1 | 1 + .../licenses/httpclient-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-4.5.8.jar.sha1 | 1 - .../licenses/httpcore-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-4.4.12.jar.sha1 | 1 + .../licenses/httpclient-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-4.5.8.jar.sha1 | 1 - .../licenses/httpcore-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-4.4.12.jar.sha1 | 1 + .../licenses/httpclient-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-4.5.8.jar.sha1 | 1 - .../licenses/httpcore-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-4.4.12.jar.sha1 | 1 + .../core/licenses/httpclient-4.5.10.jar.sha1 | 1 + .../core/licenses/httpclient-4.5.8.jar.sha1 | 1 - .../core/licenses/httpcore-4.4.11.jar.sha1 | 1 - .../core/licenses/httpcore-4.4.12.jar.sha1 | 1 + .../licenses/httpcore-nio-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-nio-4.4.12.jar.sha1 | 1 + .../licenses/httpclient-cache-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-cache-4.5.8.jar.sha1 | 1 - .../licenses/httpclient-4.5.10.jar.sha1 | 1 + .../licenses/httpclient-4.5.8.jar.sha1 | 1 - .../licenses/httpcore-4.4.11.jar.sha1 | 1 - .../licenses/httpcore-4.4.12.jar.sha1 | 1 + 47 files changed, 57 insertions(+), 41 deletions(-) create mode 100644 client/rest/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 client/rest/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 client/rest/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 client/rest/licenses/httpcore-4.4.12.jar.sha1 delete mode 100644 client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 create mode 100644 client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 create mode 100644 client/sniffer/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 client/sniffer/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 client/sniffer/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 client/sniffer/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 plugins/repository-s3/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/httpcore-4.4.12.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/httpcore-nio-4.4.12.jar.sha1 create mode 100644 x-pack/plugin/security/licenses/httpclient-cache-4.5.10.jar.sha1 delete mode 100644 x-pack/plugin/security/licenses/httpclient-cache-4.5.8.jar.sha1 create mode 100644 x-pack/snapshot-tool/licenses/httpclient-4.5.10.jar.sha1 delete mode 100644 x-pack/snapshot-tool/licenses/httpclient-4.5.8.jar.sha1 delete mode 100644 x-pack/snapshot-tool/licenses/httpcore-4.4.11.jar.sha1 create mode 100644 x-pack/snapshot-tool/licenses/httpcore-4.4.12.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 12ff2f8cda73e..386fb669f9493 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -32,8 +32,8 @@ bouncycastle = 1.61 # test dependencies randomizedrunner = 2.7.1 junit = 4.12 -httpclient = 4.5.8 -httpcore = 4.4.11 +httpclient = 4.5.10 +httpcore = 4.4.12 httpasyncclient = 4.1.4 commonslogging = 1.1.3 commonscodec = 1.11 diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java index c7e3a8e0edabb..595a055f27677 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndexLifecycleClient.java @@ -496,9 +496,9 @@ public GetSnapshotLifecycleStatsResponse getSnapshotLifecycleStats(GetSnapshotLi * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion */ - public void getSnapshotLifecycleStatsAsync(GetSnapshotLifecycleStatsRequest request, RequestOptions options, + public Cancellable getSnapshotLifecycleStatsAsync(GetSnapshotLifecycleStatsRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::getSnapshotLifecycleStats, + return restHighLevelClient.performRequestAsyncAndParseEntity(request, IndexLifecycleRequestConverters::getSnapshotLifecycleStats, options, GetSnapshotLifecycleStatsResponse::fromXContent, listener, emptySet()); } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java index a25396af648dd..ef2e9642c9899 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/SecurityClient.java @@ -1013,7 +1013,7 @@ public Cancellable invalidateApiKeyAsync(final InvalidateApiKeyRequest request, * authenticated TLS session, and it is validated by the PKI realms with {@code delegation.enabled} toggled to {@code true}.
* See the * docs for more details. - * + * * @param request the request containing the certificate chain * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @return the response from the delegate-pki-authentication API key call @@ -1031,14 +1031,14 @@ public DelegatePkiAuthenticationResponse delegatePkiAuthentication(DelegatePkiAu * {@code true}.
* See the * docs for more details. - * + * * @param request the request containing the certificate chain * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized * @param listener the listener to be notified upon request completion */ - public void delegatePkiAuthenticationAsync(DelegatePkiAuthenticationRequest request, RequestOptions options, + public Cancellable delegatePkiAuthenticationAsync(DelegatePkiAuthenticationRequest request, RequestOptions options, ActionListener listener) { - restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::delegatePkiAuthentication, options, + return restHighLevelClient.performRequestAsyncAndParseEntity(request, SecurityRequestConverters::delegatePkiAuthentication, options, DelegatePkiAuthenticationResponse::fromXContent, listener, emptySet()); } } diff --git a/client/rest/licenses/httpclient-4.5.10.jar.sha1 b/client/rest/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/client/rest/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/client/rest/licenses/httpclient-4.5.8.jar.sha1 b/client/rest/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/client/rest/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-4.4.11.jar.sha1 b/client/rest/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/client/rest/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/client/rest/licenses/httpcore-4.4.12.jar.sha1 b/client/rest/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/client/rest/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 b/client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 deleted file mode 100644 index 9e8777cb3da1c..0000000000000 --- a/client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7d0a97d01d39cff9aa3e6db81f21fddb2435f4e6 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 b/client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..4de932dc5aca0 --- /dev/null +++ b/client/rest/licenses/httpcore-nio-4.4.12.jar.sha1 @@ -0,0 +1 @@ +84cd29eca842f31db02987cfedea245af020198b \ No newline at end of file diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java index 38317af790ae7..9a893fa58a287 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientMultipleHostsIntegTests.java @@ -74,6 +74,7 @@ public static void startHttpServer() throws Exception { int numHttpServers = randomIntBetween(2, 4); httpServers = new HttpServer[numHttpServers]; httpHosts = new HttpHost[numHttpServers]; + waitForCancelHandler = new WaitForCancelHandler(); for (int i = 0; i < numHttpServers; i++) { HttpServer httpServer = createHttpServer(); httpServers[i] = httpServer; @@ -98,24 +99,30 @@ private static HttpServer createHttpServer() throws Exception { for (int statusCode : getAllStatusCodes()) { httpServer.createContext(pathPrefix + "/" + statusCode, new ResponseHandler(statusCode)); } - waitForCancelHandler = new WaitForCancelHandler(); httpServer.createContext(pathPrefix + "/wait", waitForCancelHandler); return httpServer; } private static class WaitForCancelHandler implements HttpHandler { - private CountDownLatch cancelHandlerLatch; + private volatile CountDownLatch requestCameInLatch; + private volatile CountDownLatch cancelHandlerLatch; void reset() { cancelHandlerLatch = new CountDownLatch(1); + requestCameInLatch = new CountDownLatch(1); } void cancelDone() { cancelHandlerLatch.countDown(); } + void awaitRequest() throws InterruptedException { + requestCameInLatch.await(); + } + @Override public void handle(HttpExchange exchange) throws IOException { + requestCameInLatch.countDown(); try { cancelHandlerLatch.await(); } catch (InterruptedException ignore) { @@ -226,15 +233,12 @@ public void onFailure(Exception exception) { public void testCancelAsyncRequests() throws Exception { int numRequests = randomIntBetween(5, 20); - final CountDownLatch latch = new CountDownLatch(numRequests); final List responses = new CopyOnWriteArrayList<>(); final List exceptions = new CopyOnWriteArrayList<>(); for (int i = 0; i < numRequests; i++) { + CountDownLatch latch = new CountDownLatch(1); waitForCancelHandler.reset(); - final String method = RestClientTestUtil.randomHttpMethod(getRandom()); - //we don't test status codes that are subject to retries as they interfere with hosts being stopped - final int statusCode = randomBoolean() ? randomOkStatusCode(getRandom()) : randomErrorNoRetryStatusCode(getRandom()); - Cancellable cancellable = restClient.performRequestAsync(new Request(method, "/" + statusCode), new ResponseListener() { + Cancellable cancellable = restClient.performRequestAsync(new Request("GET", "/wait"), new ResponseListener() { @Override public void onSuccess(Response response) { responses.add(response); @@ -247,10 +251,15 @@ public void onFailure(Exception exception) { latch.countDown(); } }); + if (randomBoolean()) { + //we wait for the request to get to the server-side otherwise we almost always cancel + // the request artificially on the client-side before even sending it + waitForCancelHandler.awaitRequest(); + } cancellable.cancel(); waitForCancelHandler.cancelDone(); + assertTrue(latch.await(5, TimeUnit.SECONDS)); } - assertTrue(latch.await(5, TimeUnit.SECONDS)); assertEquals(0, responses.size()); assertEquals(numRequests, exceptions.size()); for (Exception exception : exceptions) { diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java index c47fa55c9fe67..9c7c777fe1431 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostIntegTests.java @@ -105,7 +105,7 @@ private HttpServer createHttpServer() throws Exception { return httpServer; } - private class WaitForCancelHandler implements HttpHandler { + private static class WaitForCancelHandler implements HttpHandler { private final CountDownLatch cancelHandlerLatch = new CountDownLatch(1); @@ -259,6 +259,8 @@ public void onFailure(Exception exception) { /** * This test verifies some assumptions that we rely upon around the way the async http client works when reusing the same request * throughout multiple retries, and the use of the {@link HttpRequestBase#abort()} method. + * In fact the low-level REST client reuses the same request instance throughout multiple retries, and relies on the http client + * to set the future ref to the request properly so that when abort is called, the proper future gets cancelled. */ public void testRequestResetAndAbort() throws Exception { try (CloseableHttpAsyncClient client = HttpAsyncClientBuilder.create().build()) { @@ -273,10 +275,15 @@ public void testRequestResetAndAbort() throws Exception { { httpGet.reset(); assertFalse(httpGet.isAborted()); - httpGet.abort();//this has no effect on the next call (although isAborted will return true until the next reset) + httpGet.abort(); Future future = client.execute(httpHost, httpGet, null); - assertEquals(200, future.get().getStatusLine().getStatusCode()); - assertFalse(future.isCancelled()); + try { + future.get(); + fail("expected cancellation exception"); + } catch(CancellationException e) { + //expected + } + assertTrue(future.isCancelled()); } { httpGet.reset(); diff --git a/client/sniffer/licenses/httpclient-4.5.10.jar.sha1 b/client/sniffer/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/client/sniffer/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/client/sniffer/licenses/httpclient-4.5.8.jar.sha1 b/client/sniffer/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/client/sniffer/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore-4.4.11.jar.sha1 b/client/sniffer/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/client/sniffer/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore-4.4.12.jar.sha1 b/client/sniffer/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/client/sniffer/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpclient-4.5.10.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpclient-4.5.8.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/plugins/discovery-azure-classic/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpclient-4.5.10.jar.sha1 b/plugins/discovery-ec2/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/plugins/discovery-ec2/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpclient-4.5.8.jar.sha1 b/plugins/discovery-ec2/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/plugins/discovery-ec2/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/plugins/discovery-ec2/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpclient-4.5.10.jar.sha1 b/plugins/discovery-gce/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/plugins/discovery-gce/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpclient-4.5.8.jar.sha1 b/plugins/discovery-gce/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/plugins/discovery-gce/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/plugins/discovery-gce/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpclient-4.5.10.jar.sha1 b/plugins/repository-gcs/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/plugins/repository-gcs/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpclient-4.5.8.jar.sha1 b/plugins/repository-gcs/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/plugins/repository-gcs/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 b/plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpcore-4.4.12.jar.sha1 b/plugins/repository-gcs/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/plugins/repository-gcs/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpclient-4.5.10.jar.sha1 b/plugins/repository-s3/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/plugins/repository-s3/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpclient-4.5.8.jar.sha1 b/plugins/repository-s3/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/plugins/repository-s3/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/plugins/repository-s3/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpclient-4.5.10.jar.sha1 b/x-pack/plugin/core/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/x-pack/plugin/core/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpclient-4.5.8.jar.sha1 b/x-pack/plugin/core/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/x-pack/plugin/core/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-4.4.12.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/x-pack/plugin/core/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 deleted file mode 100644 index 9e8777cb3da1c..0000000000000 --- a/x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7d0a97d01d39cff9aa3e6db81f21fddb2435f4e6 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-nio-4.4.12.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-nio-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..4de932dc5aca0 --- /dev/null +++ b/x-pack/plugin/core/licenses/httpcore-nio-4.4.12.jar.sha1 @@ -0,0 +1 @@ +84cd29eca842f31db02987cfedea245af020198b \ No newline at end of file diff --git a/x-pack/plugin/security/licenses/httpclient-cache-4.5.10.jar.sha1 b/x-pack/plugin/security/licenses/httpclient-cache-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..3b1d8db643114 --- /dev/null +++ b/x-pack/plugin/security/licenses/httpclient-cache-4.5.10.jar.sha1 @@ -0,0 +1 @@ +b195778247a21e980cb9f80c41364dc0c38feaef \ No newline at end of file diff --git a/x-pack/plugin/security/licenses/httpclient-cache-4.5.8.jar.sha1 b/x-pack/plugin/security/licenses/httpclient-cache-4.5.8.jar.sha1 deleted file mode 100644 index 87db7aba09e2c..0000000000000 --- a/x-pack/plugin/security/licenses/httpclient-cache-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -bb984b73da2153285b660f3e278498abd94ccbb5 \ No newline at end of file diff --git a/x-pack/snapshot-tool/licenses/httpclient-4.5.10.jar.sha1 b/x-pack/snapshot-tool/licenses/httpclient-4.5.10.jar.sha1 new file mode 100644 index 0000000000000..b708efd0dd57f --- /dev/null +++ b/x-pack/snapshot-tool/licenses/httpclient-4.5.10.jar.sha1 @@ -0,0 +1 @@ +7ca2e4276f4ef95e4db725a8cd4a1d1e7585b9e5 \ No newline at end of file diff --git a/x-pack/snapshot-tool/licenses/httpclient-4.5.8.jar.sha1 b/x-pack/snapshot-tool/licenses/httpclient-4.5.8.jar.sha1 deleted file mode 100644 index 73f0d30c7093b..0000000000000 --- a/x-pack/snapshot-tool/licenses/httpclient-4.5.8.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c27c9d6f15435dc2b6947112027b418b0eef32b9 \ No newline at end of file diff --git a/x-pack/snapshot-tool/licenses/httpcore-4.4.11.jar.sha1 b/x-pack/snapshot-tool/licenses/httpcore-4.4.11.jar.sha1 deleted file mode 100644 index 6d64372bfccd8..0000000000000 --- a/x-pack/snapshot-tool/licenses/httpcore-4.4.11.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/x-pack/snapshot-tool/licenses/httpcore-4.4.12.jar.sha1 b/x-pack/snapshot-tool/licenses/httpcore-4.4.12.jar.sha1 new file mode 100644 index 0000000000000..3c046171b30da --- /dev/null +++ b/x-pack/snapshot-tool/licenses/httpcore-4.4.12.jar.sha1 @@ -0,0 +1 @@ +21ebaf6d532bc350ba95bd81938fa5f0e511c132 \ No newline at end of file