You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/03/16 20:24:31 UTC

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

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 <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7023/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Compilation Failed
https://goo.gl/wsHNrq : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/3550/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3062/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/2498/4/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java:

PS4, Line 196: 
> can remove if we stick to collections
Done


PS4, Line 198:         List<Future<Void>> tasks = new ArrayList<>(operatorNodePushablesBFSOrder.size());
             :         Queue<Throwable> failures = new ArrayBlockingQueue<>(operatorNodePushablesB
> why not leave as lists?  we can then remove the use of indexes and just ope
Done


https://asterix-gerrit.ics.uci.edu/#/c/2498/5/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java:

PS5, Line 205: current
> don't need the index
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3036/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/3540/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has abandoned this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Abandoned

changed branch

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: abandon
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Michael Blow (Code Review)" <do...@asterixdb.incubator.apache.org>.
Michael Blow has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3816/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2498

to look at the new patch set (#6).

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, 19 insertions(+), 6 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/98/2498/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2908/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/3539/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Analytics Compatibility Compilation Successful
https://goo.gl/GzvKFu : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3074/ (9/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3073/ (9/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7048/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3800/ (10/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3003/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1097/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1105/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3808/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Compilation Failed
https://goo.gl/gfNbEa : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3612/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5966/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2986/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3086/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/VE4ZvA : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/UQQQHT : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7022/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1083/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1080/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/mKcLjc : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2498

to look at the new patch set (#3).

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, 26 insertions(+), 11 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/98/2498/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8530/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3630/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3292/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/3518/ (11/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8533/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3011/ (10/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/ieBw59 : UNSTABLE

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2908/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8556/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2910/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/3542/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/eDKfhh : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3293/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3059/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2498

to look at the new patch set (#8).

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
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
2 files changed, 46 insertions(+), 41 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/98/2498/8
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3605/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3290/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3068/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7030/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3034/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3607/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3060/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3042/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2985/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3291/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3298/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3316/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Analytics Compatibility Compilation Successful
https://goo.gl/CxhGVd : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8548/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Analytics Compatibility Compilation Successful
https://goo.gl/QxQWKE : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5974/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2910/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1088/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3035/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3051/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3063/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3041/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3090/ (9/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3792/ (10/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/3515/ (11/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3078/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3308/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/3564/ (11/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3604/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3087/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Analytics Compatibility Compilation Successful
https://goo.gl/V8YBKq : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://goo.gl/Qarzoy : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7025/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3606/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3072/ (9/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2498

to look at the new patch set (#7).

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
M hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java
2 files changed, 49 insertions(+), 41 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/98/2498/7
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5974/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1087/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5992/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8531/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/2498

to look at the new patch set (#5).

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, 22 insertions(+), 8 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/98/2498/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/3558/ (12/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/3526/ (11/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1082/ (6/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8532/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/niEyN8 : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7024/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2988/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2498/5/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java:

PS5, Line 205: current
don't need the index


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Michael Blow (Code Review)" <do...@asterixdb.incubator.apache.org>.
Michael Blow has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2498/7/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java:

PS7, Line 228:             Throwable next = failures.poll();
             :             while (next != null) {
             :                 ExceptionUtils.suppress(root, next);
             :                 next = failures.poll();
             :             }
replace with a one-liner? e.g.

    failures.forEach(f -> ExceptionUtils.suppress(root, f))


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7040/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2498/7/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java:

PS7, Line 228:             Throwable next = failures.poll();
             :             while (next != null) {
             :                 ExceptionUtils.suppress(root, next);
             :                 next = failures.poll();
             :             }
> replace with a one-liner? e.g.
Done


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2916/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Analytics Compatibility Compilation Successful
https://goo.gl/FdaXSZ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2916/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Michael Blow (Code Review)" <do...@asterixdb.incubator.apache.org>.
Michael Blow has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/2498/4/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java:

PS4, Line 196:     @SuppressWarnings("unchecked")
can remove if we stick to collections


PS4, Line 198:         Future<Void>[] tasks = new Future[operatorNodePushablesBFSOrder.size()];
             :         Throwable[] failures = new Throwable[operatorNodePushablesBFSOrder.size()];
why not leave as lists?  we can then remove the use of indexes and just operate on the collections as before?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3061/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8538/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3079/ (9/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7031/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5992/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/3534/ (11/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2994/ (5/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2987/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5963/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3613/ (7/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3788/ (10/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1081/ (4/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3033/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3100/ (9/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5966/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2993/ (2/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8539/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/3299/ (3/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 4:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2498/4/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
File hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java:

PS4, Line 198:         Future<Void>[] tasks = new Future[operatorNodePushablesBFSOrder.size()];
             :         Throwable[] failures = new Throwable[operatorNodePushablesBFSOrder.size()];
> why not leave as lists?  we can then remove the use of indexes and just ope
Somehow I think this is better because I need a mechanism for all tasks to report their failures. using the indexes removes the need for synchronization, otherwise, we will have to use a blocking list

Either way, I don't have a strong opinion.
Thoughts?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/3622/ (1/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNod...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [NO ISSUE][RT] Report all errors on SuperActivityOperatorNodePushable
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3069/ (8/12)

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2498
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ibbf31dd91303ce2f606734fcccb19270875266b3
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No