Java API client version
main at commit 056ed9c30
Java version
OpenJDK 25.0.2
Elasticsearch Version
N/A. This is a client-side transport issue and reproduces against all elasticsearch versions.
Problem description
Hi!
Synchronous Java API client calls keep the underlying HTTP request running after the calling thread is interrupted. The result is discarded on the client side, but this prevents ElasticSearch from cancelling expensive searches that have been interrupted by the caller.
I initially reported this issue to Trino here: trinodb/trino#28927. But Trino is probably not the only one affected by this, and I think a fix in the ElasticSearch client would be nice.
Expected behavior: Interrupting a thread blocked in a synchronous client call should abort the in-flight HTTP request.
Actual behavior: The calling thread returns after interruption, but the underlying request continues running in the background.
Relevant code:
- Rest5 sync path waits on
client.execute(...).get() without cancelling on interrupt:
|
httpResponse = client.execute(context.requestProducer, |
|
context.asyncResponseConsumer, |
|
context.context, null).get(); |
- The cancellable future returned by execute is immediately awaited and the cancellation handle is lost
Steps to reproduce
- Create a synchronous
ElasticsearchClient
- Run an expensive elasticsearch operation in one thread
- Wait until the request reaches the server, then interrupt the client thread.
- Observe whether the server cancels the task associated to the request
Here is a minimal repro: https://github.com/lovasoa/elasticsearch-java/blob/a5598215482233ceee4c44d6dc6642c6b83e0f23/java-client/src/test/java/co/elastic/clients/transport/InterruptSyncRequestTest.java
Reproduction.java
CountDownLatch requestArrived = new CountDownLatch(1);
CompletableFuture<IOException> serverException = new CompletableFuture<>();
HttpServer httpServer = createSlowServer(requestArrived, serverException);
httpServer.start();
try {
ElasticsearchClient client = ElasticsearchTestClient.createClient(httpServer, null);
Thread clientThread = new Thread(() -> {
try {
client.count();
} catch (IOException | ElasticsearchException e) {
// ignored for repro
}
});
clientThread.start();
assertTrue(requestArrived.await(5, TimeUnit.SECONDS));
clientThread.interrupt();
// Expected if the sync request is really cancelled:
assertInstanceOf(IOException.class, serverException.get(5, TimeUnit.SECONDS));
} finally {
httpServer.stop(0);
}
For comparison, cancelling the async future does close the connection in the companion test:
CompletableFuture<?> future = asyncClient.count();
assertTrue(requestArrived.await(5, TimeUnit.SECONDS));
future.cancel(true);
assertInstanceOf(IOException.class, serverException.get(5, TimeUnit.SECONDS));
Java API client version
mainat commit056ed9c30Java version
OpenJDK 25.0.2
Elasticsearch Version
N/A. This is a client-side transport issue and reproduces against all elasticsearch versions.
Problem description
Hi!
Synchronous Java API client calls keep the underlying HTTP request running after the calling thread is interrupted. The result is discarded on the client side, but this prevents ElasticSearch from cancelling expensive searches that have been interrupted by the caller.
I initially reported this issue to Trino here: trinodb/trino#28927. But Trino is probably not the only one affected by this, and I think a fix in the ElasticSearch client would be nice.
Expected behavior: Interrupting a thread blocked in a synchronous client call should abort the in-flight HTTP request.
Actual behavior: The calling thread returns after interruption, but the underlying request continues running in the background.
Relevant code:
client.execute(...).get()without cancelling on interrupt:elasticsearch-java/rest5-client/src/main/java/co/elastic/clients/transport/rest5_client/low_level/Rest5Client.java
Lines 302 to 304 in 056ed9c
Steps to reproduce
ElasticsearchClientHere is a minimal repro: https://github.com/lovasoa/elasticsearch-java/blob/a5598215482233ceee4c44d6dc6642c6b83e0f23/java-client/src/test/java/co/elastic/clients/transport/InterruptSyncRequestTest.java
Reproduction.java
For comparison, cancelling the async future does close the connection in the companion test: