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/27 04:10:59 UTC

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

abdullah alamoudi has uploaded a new change for review.

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................

[NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- Primary upsert operator uses an IFrameOperationCallback
  to perform per frame operations. In this change,
  IFrameOperationCallback extends Closeable and allow the
  callback to release resources on the close of the pipeline.

Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
5 files changed, 30 insertions(+), 13 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/36/2536/1

diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index 9ec13ef..b7af0b6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -74,4 +74,8 @@
         dsInfo.untouch();
         dsInfo.getIndexes().get(resourceId).untouch();
     }
+
+    public DatasetInfo getDatasetInfo() {
+        return dsInfo;
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
index 5e3879f..8f28752 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.common.dataflow;
 
+import java.io.IOException;
+
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallback;
@@ -43,5 +45,10 @@
         public void frameCompleted() throws HyracksDataException {
             // No Op
         }
+
+        @Override
+        public void close() throws IOException {
+            // No Op
+        }
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
index e33dfb6..029f82c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -243,6 +244,11 @@
                     callback.frameCompleted();
                     appender.write(writer, true);
                 }
+
+                @Override
+                public void close() throws IOException {
+                    callback.close();
+                }
             };
         } catch (Throwable e) { // NOSONAR: Re-thrown
             throw HyracksDataException.create(e);
@@ -360,16 +366,12 @@
 
     @Override
     public void close() throws HyracksDataException {
-        try {
-            try {
-                if (cursor != null) {
-                    cursor.destroy();
-                }
-            } finally {
-                writer.close();
-            }
-        } finally {
-            indexHelper.close();
+        Throwable failure = CleanupUtils.close(frameOpCallback, null);
+        failure = CleanupUtils.destroy(failure, cursor);
+        failure = CleanupUtils.close(writer, failure);
+        failure = CleanupUtils.close(indexHelper, failure);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
index be10b27..9e65eb2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
@@ -18,11 +18,13 @@
  */
 package org.apache.hyracks.storage.am.common.api;
 
+import java.io.Closeable;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.LocalResource;
 
-public interface IIndexDataflowHelper {
+public interface IIndexDataflowHelper extends Closeable {
 
     /**
      * If open throws an exception, it means that the index was not opened successfully.
@@ -36,6 +38,7 @@
      *
      * @throws HyracksDataException
      */
+    @Override
     void close() throws HyracksDataException;
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
index 96eb559..df78c53 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
@@ -18,13 +18,14 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.api;
 
+import java.io.Closeable;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An interface that is used to enable frame level operation on indexes
  */
-@FunctionalInterface
-public interface IFrameOperationCallback {
+public interface IFrameOperationCallback extends Closeable {
     /**
      * Called once processing the frame is done before calling nextFrame on the next IFrameWriter in
      * the pipeline

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2: Contrib+1

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has submitted this change and it was merged.

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


[NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- Primary upsert operator uses an IFrameOperationCallback
  to perform per frame operations. In this change,
  IFrameOperationCallback extends Closeable and allow the
  callback to release resources on the close of the pipeline.

Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2536
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <mh...@apache.org>
---
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
M hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
5 files changed, 30 insertions(+), 13 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Jenkins: Verified; No violations found; ; Verified
  Murtadha Hubail: Looks good to me, approved



diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index 9ec13ef..b7af0b6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -74,4 +74,8 @@
         dsInfo.untouch();
         dsInfo.getIndexes().get(resourceId).untouch();
     }
+
+    public DatasetInfo getDatasetInfo() {
+        return dsInfo;
+    }
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
index 5e3879f..8f28752 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/NoOpFrameOperationCallbackFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.common.dataflow;
 
+import java.io.IOException;
+
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallback;
@@ -43,5 +45,10 @@
         public void frameCompleted() throws HyracksDataException {
             // No Op
         }
+
+        @Override
+        public void close() throws IOException {
+            // No Op
+        }
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
index e58a7db..68053d3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMPrimaryUpsertOperatorNodePushable.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -243,6 +244,11 @@
                     callback.frameCompleted();
                     appender.write(writer, true);
                 }
+
+                @Override
+                public void close() throws IOException {
+                    callback.close();
+                }
             };
         } catch (Throwable e) { // NOSONAR: Re-thrown
             throw HyracksDataException.create(e);
@@ -360,16 +366,12 @@
 
     @Override
     public void close() throws HyracksDataException {
-        try {
-            try {
-                if (cursor != null) {
-                    cursor.destroy();
-                }
-            } finally {
-                writer.close();
-            }
-        } finally {
-            indexHelper.close();
+        Throwable failure = CleanupUtils.close(frameOpCallback, null);
+        failure = CleanupUtils.destroy(failure, cursor);
+        failure = CleanupUtils.close(writer, failure);
+        failure = CleanupUtils.close(indexHelper, failure);
+        if (failure != null) {
+            throw HyracksDataException.create(failure);
         }
     }
 
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
index be10b27..9e65eb2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IIndexDataflowHelper.java
@@ -18,11 +18,13 @@
  */
 package org.apache.hyracks.storage.am.common.api;
 
+import java.io.Closeable;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.common.IIndex;
 import org.apache.hyracks.storage.common.LocalResource;
 
-public interface IIndexDataflowHelper {
+public interface IIndexDataflowHelper extends Closeable {
 
     /**
      * If open throws an exception, it means that the index was not opened successfully.
@@ -36,6 +38,7 @@
      *
      * @throws HyracksDataException
      */
+    @Override
     void close() throws HyracksDataException;
 
     /**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
index 96eb559..df78c53 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IFrameOperationCallback.java
@@ -18,13 +18,14 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.api;
 
+import java.io.Closeable;
+
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An interface that is used to enable frame level operation on indexes
  */
-@FunctionalInterface
-public interface IFrameOperationCallback {
+public interface IFrameOperationCallback extends Closeable {
     /**
      * Called once processing the frame is done before calling nextFrame on the next IFrameWriter in
      * the pipeline

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

Gerrit-MessageType: merged
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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

Change subject: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[release-0.9.4-pre-rc]: [NO ISSUE][RT] Inherit Closeable in IFrameOperationCallback

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] Inherit Closeable in IFrameOperationCallback
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iaacd6538c27d5498868256616a793e0ebcec3729
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: release-0.9.4-pre-rc
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-HasComments: No