leventov commented on a change in pull request #8700: New Thread for Cancel
URL: https://github.com/apache/incubator-druid/pull/8700#discussion_r341482446
##########
File path: server/src/main/java/org/apache/druid/client/DirectDruidClient.java
##########
@@ -465,38 +467,31 @@ public void onFailure(Throwable t)
{
openConnections.getAndDecrement();
if (future.isCancelled()) {
- // forward the cancellation to underlying queriable node
- try {
- StatusResponseHolder res = httpClient.go(
- new Request(
- HttpMethod.DELETE,
- new URL(cancelUrl)
- ).setContent(objectMapper.writeValueAsBytes(query))
- .setHeader(
- HttpHeaders.Names.CONTENT_TYPE,
- isSmile
- ? SmileMediaTypes.APPLICATION_JACKSON_SMILE
- : MediaType.APPLICATION_JSON
- ),
- StatusResponseHandler.getInstance(),
- Duration.standardSeconds(1)
- ).get(1, TimeUnit.SECONDS);
-
- if (res.getStatus().getCode() >= 500) {
- throw new RE(
- "Error cancelling query[%s]: queriable node returned
status[%d] [%s].",
- query,
- res.getStatus().getCode(),
- res.getStatus().getReasonPhrase()
- );
+ Runnable cancelRunnable = () -> {
+ try {
+ httpClient.go(
+ new Request(HttpMethod.DELETE, new URL(cancelUrl))
+ .setContent(objectMapper.writeValueAsBytes(query))
+ .setHeader(HttpHeaders.Names.CONTENT_TYPE, isSmile ?
SmileMediaTypes.APPLICATION_JACKSON_SMILE : MediaType.APPLICATION_JSON),
+ StatusResponseHandler.getInstance(),
+ Duration.standardSeconds(1));
}
- }
- catch (IOException | ExecutionException | InterruptedException
| TimeoutException e) {
- throw new RuntimeException(e);
- }
+ catch (IOException e) {
+ log.error("Error cancelling query[%s]: ", query);
+ }
+ };
+
+ Future<?> responseFuture =
queryCancellationExecutor.submit(cancelRunnable);
+ Runnable checkRunnable = () -> {
+ if (!responseFuture.isDone()) {
Review comment:
It's a wrong future. You should check the future returned from
`httpClient.go()`, from inside `cancelRunnable`. There is a difference when
several cancellation requests are scheduled for `queryCancellationExecutor`. We
want to start count 1 second from `httpClient.go()` moment, not considering the
time this task waits in the queue of `queryCancellationExecutor`.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]