|
@@ -13,9 +13,8 @@ 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;
|
|
|
+import java.util.concurrent.Semaphore;
|
|
|
|
|
|
/**
|
|
|
* {@code FailureCollector} is responsible for collecting exceptions that occur in the compute engine.
|
|
@@ -26,12 +25,11 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|
|
*/
|
|
|
public final class FailureCollector {
|
|
|
private final Queue<Exception> cancelledExceptions = ConcurrentCollections.newQueue();
|
|
|
- private final AtomicInteger cancelledExceptionsCount = new AtomicInteger();
|
|
|
+ private final Semaphore cancelledExceptionsPermits;
|
|
|
|
|
|
private final Queue<Exception> nonCancelledExceptions = ConcurrentCollections.newQueue();
|
|
|
- private final AtomicInteger nonCancelledExceptionsCount = new AtomicInteger();
|
|
|
+ private final Semaphore nonCancelledExceptionsPermits;
|
|
|
|
|
|
- private final int maxExceptions;
|
|
|
private volatile boolean hasFailure = false;
|
|
|
private Exception finalFailure = null;
|
|
|
|
|
@@ -43,7 +41,8 @@ public final class FailureCollector {
|
|
|
if (maxExceptions <= 0) {
|
|
|
throw new IllegalArgumentException("maxExceptions must be at least one");
|
|
|
}
|
|
|
- this.maxExceptions = maxExceptions;
|
|
|
+ this.cancelledExceptionsPermits = new Semaphore(maxExceptions);
|
|
|
+ this.nonCancelledExceptionsPermits = new Semaphore(maxExceptions);
|
|
|
}
|
|
|
|
|
|
private static Exception unwrapTransportException(TransportException te) {
|
|
@@ -60,13 +59,12 @@ public final class FailureCollector {
|
|
|
public void unwrapAndCollect(Exception e) {
|
|
|
e = e instanceof TransportException te ? unwrapTransportException(te) : e;
|
|
|
if (ExceptionsHelper.unwrap(e, TaskCancelledException.class) != null) {
|
|
|
- if (cancelledExceptionsCount.incrementAndGet() <= maxExceptions) {
|
|
|
+ if (nonCancelledExceptions.isEmpty() && cancelledExceptionsPermits.tryAcquire()) {
|
|
|
cancelledExceptions.add(e);
|
|
|
}
|
|
|
- } else {
|
|
|
- if (nonCancelledExceptionsCount.incrementAndGet() <= maxExceptions) {
|
|
|
- nonCancelledExceptions.add(e);
|
|
|
- }
|
|
|
+ } else if (nonCancelledExceptionsPermits.tryAcquire()) {
|
|
|
+ nonCancelledExceptions.add(e);
|
|
|
+ cancelledExceptions.clear();
|
|
|
}
|
|
|
hasFailure = true;
|
|
|
}
|
|
@@ -99,20 +97,22 @@ public final class FailureCollector {
|
|
|
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;
|
|
|
- }
|
|
|
+ for (Exception e : nonCancelledExceptions) {
|
|
|
+ if (first == null) {
|
|
|
+ first = e;
|
|
|
+ } else if (first != e) {
|
|
|
+ first.addSuppressed(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (first != null) {
|
|
|
+ return first;
|
|
|
+ }
|
|
|
+ for (Exception e : cancelledExceptions) {
|
|
|
+ if (first == null) {
|
|
|
+ first = e;
|
|
|
+ } else if (first != e) {
|
|
|
+ first.addSuppressed(e);
|
|
|
}
|
|
|
}
|
|
|
assert first != null;
|