abdullah alamoudi has uploaded a new change for review.
https://asterix-gerrit.ics.uci.edu/2498
Change subject: [NO ISSUE][RT] Report all errors on
SuperActivityOperatorNodePushable
......................................................................
[NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
- user model changes: no
- storage format changes: no
- interface changes: no
Details:
- Currently, if a failure happens in SuperActivityOperatorNodePushable,
we only report that failure and miss the rest of the failures.
This is especially critical in case of job cancellation since we
don't know where each thread was interrupted.
- After this change, we suppress all other failures in the root
failure for reporting purposes.
Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
---
M
hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
1 file changed, 27 insertions(+), 8 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb
refs/changes/98/2498/1
diff --git
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
index 83ab532..66b96ce 100644
---
a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
+++
b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -32,6 +32,7 @@
import java.util.concurrent.Semaphore;
import org.apache.commons.collections4.MapUtils;
+import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -44,6 +45,7 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobFlag;
+import org.apache.hyracks.api.util.ExceptionUtils;
/**
* The runtime of a SuperActivity, which internally executes a DAG of
one-to-one
@@ -193,15 +195,26 @@
}
private void runInParallel(OperatorNodePushableAction action) throws
HyracksDataException {
- List<Future<Void>> tasks = new ArrayList<>();
+ List<Future<Void>> tasks = new
ArrayList<>(operatorNodePushablesBFSOrder.size());
+ List<MutableObject<Throwable>> failures = new
ArrayList<>(operatorNodePushablesBFSOrder.size());
final Semaphore startSemaphore = new Semaphore(1 -
operatorNodePushablesBFSOrder.size());
final Semaphore completeSemaphore = new Semaphore(1 -
operatorNodePushablesBFSOrder.size());
+ int completed = 0;
+ Throwable root = null;
try {
- for (final IOperatorNodePushable op :
operatorNodePushablesBFSOrder) {
+
+ for (int i = 0; i < operatorNodePushablesBFSOrder.size(); i++) {
+ failures.add(new MutableObject<Throwable>());
+ }
+ for (int i = 0; i < operatorNodePushablesBFSOrder.size(); i++) {
+ final int current = i;
tasks.add(ctx.getExecutorService().submit(() -> {
startSemaphore.release();
try {
- action.run(op);
+ action.run(operatorNodePushablesBFSOrder.get(current));
+ } catch (Throwable th) { // NOSONAR: Must catch all causes
of failure
+ failures.get(current).setValue(th);
+ throw th;
} finally {
completeSemaphore.release();
}
@@ -210,14 +223,20 @@
}
for (Future<Void> task : tasks) {
task.get();
+ completed++;
}
- } catch (InterruptedException e) {
- cancelTasks(tasks, startSemaphore, completeSemaphore);
- Thread.currentThread().interrupt();
- throw HyracksDataException.create(e);
} catch (ExecutionException e) {
+ root = e.getCause();
+ completed++;
+ } catch (Throwable e) { // NOSONAR: Must catch all causes of failure
+ root = e;
+ }
+ if (root != null) {
cancelTasks(tasks, startSemaphore, completeSemaphore);
- throw HyracksDataException.create(e.getCause());
+ for (int i = completed; i < operatorNodePushablesBFSOrder.size();
i++) {
+ root = ExceptionUtils.suppress(root,
failures.get(i).getValue());
+ }
+ throw HyracksDataException.create(root);
}
}
--
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newchange
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <[email protected]>