-
Notifications
You must be signed in to change notification settings - Fork 24.9k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Browse files
Browse the repository at this point in the history
Currently, if a child request fails, we automatically trigger cancellation for ES|QL requests. This can result in TaskCancelledException being collected by the RefCountingListener first, which then returns that exception to the caller. For example, if we encounter a CircuitBreakingException (429), we might incorrectly return a TaskCancelledException (400) instead. This change introduces the ComputeListener, a variant of RefCountingListener, which selects the most appropriate exception to return to the caller. I also integrated the following features into ComputeListener to simplify ComputeService: - Automatic cancellation of sub-tasks on failure. - Collection of profiles from sub-tasks. - Collection of response headers from sub-tasks.
- Loading branch information
Showing
9 changed files
with
658 additions
and
228 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,5 @@ | ||
pr: 110400 | ||
summary: Introduce compute listener | ||
area: ES|QL | ||
type: bug | ||
issues: [] |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
112 changes: 112 additions & 0 deletions
112
...lugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FailureCollector.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,112 @@ | ||
/* | ||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one | ||
* or more contributor license agreements. Licensed under the Elastic License | ||
* 2.0; you may not use this file except in compliance with the Elastic License | ||
* 2.0. | ||
*/ | ||
|
||
package org.elasticsearch.compute.operator; | ||
|
||
import org.elasticsearch.ElasticsearchException; | ||
import org.elasticsearch.ExceptionsHelper; | ||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections; | ||
import org.elasticsearch.tasks.TaskCancelledException; | ||
import org.elasticsearch.transport.TransportException; | ||
|
||
import java.util.List; | ||
import java.util.Queue; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
/** | ||
* {@code FailureCollector} is responsible for collecting exceptions that occur in the compute engine. | ||
* The collected exceptions are categorized into task-cancelled and non-task-cancelled exceptions. | ||
* To limit memory usage, this class collects only the first 10 exceptions in each category by default. | ||
* When returning the accumulated failure to the caller, this class prefers non-task-cancelled exceptions | ||
* over task-cancelled ones as they are more useful for diagnosing issues. | ||
*/ | ||
public final class FailureCollector { | ||
private final Queue<Exception> cancelledExceptions = ConcurrentCollections.newQueue(); | ||
private final AtomicInteger cancelledExceptionsCount = new AtomicInteger(); | ||
|
||
private final Queue<Exception> nonCancelledExceptions = ConcurrentCollections.newQueue(); | ||
private final AtomicInteger nonCancelledExceptionsCount = new AtomicInteger(); | ||
|
||
private final int maxExceptions; | ||
private volatile boolean hasFailure = false; | ||
private Exception finalFailure = null; | ||
|
||
public FailureCollector() { | ||
this(10); | ||
} | ||
|
||
public FailureCollector(int maxExceptions) { | ||
if (maxExceptions <= 0) { | ||
throw new IllegalArgumentException("maxExceptions must be at least one"); | ||
} | ||
this.maxExceptions = maxExceptions; | ||
} | ||
|
||
public void unwrapAndCollect(Exception originEx) { | ||
final Exception e = originEx instanceof TransportException | ||
? (originEx.getCause() instanceof Exception cause ? cause : new ElasticsearchException(originEx.getCause())) | ||
: originEx; | ||
if (ExceptionsHelper.unwrap(e, TaskCancelledException.class) != null) { | ||
if (cancelledExceptionsCount.incrementAndGet() <= maxExceptions) { | ||
cancelledExceptions.add(e); | ||
} | ||
} else { | ||
if (nonCancelledExceptionsCount.incrementAndGet() <= maxExceptions) { | ||
nonCancelledExceptions.add(e); | ||
} | ||
} | ||
hasFailure = true; | ||
} | ||
|
||
/** | ||
* @return {@code true} if any failure has been collected, {@code false} otherwise | ||
*/ | ||
public boolean hasFailure() { | ||
return hasFailure; | ||
} | ||
|
||
/** | ||
* Returns the accumulated failure, preferring non-task-cancelled exceptions over task-cancelled ones. | ||
* Once this method builds the failure, incoming failures are discarded. | ||
* | ||
* @return the accumulated failure, or {@code null} if no failure has been collected | ||
*/ | ||
public Exception getFailure() { | ||
if (hasFailure == false) { | ||
return null; | ||
} | ||
synchronized (this) { | ||
if (finalFailure == null) { | ||
finalFailure = buildFailure(); | ||
} | ||
return finalFailure; | ||
} | ||
} | ||
|
||
private Exception buildFailure() { | ||
assert hasFailure; | ||
assert Thread.holdsLock(this); | ||
int total = 0; | ||
Exception first = null; | ||
for (var exceptions : List.of(nonCancelledExceptions, cancelledExceptions)) { | ||
for (Exception e : exceptions) { | ||
if (first == null) { | ||
first = e; | ||
total++; | ||
} else if (first != e) { | ||
first.addSuppressed(e); | ||
total++; | ||
} | ||
if (total >= maxExceptions) { | ||
return first; | ||
} | ||
} | ||
} | ||
assert first != null; | ||
return first; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.