You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org> on 2019/05/01 22:26:31 UTC

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Ali Alsuliman has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/3372


Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................

[ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

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

Details:
The "VariableFrameMemoryManager" releases the physical frames
objects, the logical frames objects and the free slots objects
on reset(). This patch is to retain those objects and only
release them on close() (as also stated in the interface).

- added close() to IFrameFreeSlotPolicy to make the interface similar
to the other interfaces (IFrameBufferManager, IFramePool, ... etc)
where close() is supposed to release while reset() is supposed to only
reset but retain resources.
- renamed "InMemorySortRuntimeFactory" to "MicroSortRuntimeFactory".

Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
---
M hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
R hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
M hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
10 files changed, 90 insertions(+), 40 deletions(-)



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

diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
index c6a7d9d..413c1a4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
@@ -32,7 +32,7 @@
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.sort.InMemorySortRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.sort.MicroSortRuntimeFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -80,7 +80,7 @@
             i++;
         }
 
-        IPushRuntimeFactory runtime = new InMemorySortRuntimeFactory(sortFields, nkcf, comps, null, maxNumberOfFrames);
+        IPushRuntimeFactory runtime = new MicroSortRuntimeFactory(sortFields, nkcf, comps, null, maxNumberOfFrames);
         builder.contributeMicroOperator(op, runtime, recDescriptor);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
similarity index 96%
rename from hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
rename to hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
index b34b7b9..ad44c90 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
@@ -39,7 +39,7 @@
 import org.apache.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortRunMerger;
 
-public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+public class MicroSortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
     private final int framesLimit;
@@ -47,14 +47,14 @@
     private final INormalizedKeyComputerFactory[] keyNormalizerFactories;
     private final IBinaryComparatorFactory[] comparatorFactories;
 
-    public InMemorySortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+    public MicroSortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
             IBinaryComparatorFactory[] comparatorFactories, int[] projectionList, int framesLimit) {
         this(sortFields, firstKeyNormalizerFactory != null
                 ? new INormalizedKeyComputerFactory[] { firstKeyNormalizerFactory } : null, comparatorFactories,
                 projectionList, framesLimit);
     }
 
-    public InMemorySortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory[] keyNormalizerFactories,
+    public MicroSortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory[] keyNormalizerFactories,
             IBinaryComparatorFactory[] comparatorFactories, int[] projectionList, int framesLimit) {
         super(projectionList);
         // Obs: the projection list is currently ignored.
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index e9b3fc3..aeb22b6 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -47,7 +47,7 @@
 import org.apache.hyracks.algebricks.runtime.operators.group.MicroPreClusteredGroupRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
 import org.apache.hyracks.algebricks.runtime.operators.meta.SubplanRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.sort.InMemorySortRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.sort.MicroSortRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory;
@@ -718,8 +718,8 @@
                 new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
 
         // the algebricks op.
-        InMemorySortRuntimeFactory sort =
-                new InMemorySortRuntimeFactory(new int[] { 1 }, (INormalizedKeyComputerFactory) null,
+        MicroSortRuntimeFactory sort =
+                new MicroSortRuntimeFactory(new int[] { 1 }, (INormalizedKeyComputerFactory) null,
                         new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, null, 50);
         RecordDescriptor sortDesc = scannerDesc;
 
@@ -834,8 +834,8 @@
 
         // the sort (by nation id)
         RecordDescriptor sortDesc = scannerDesc;
-        InMemorySortRuntimeFactory sort =
-                new InMemorySortRuntimeFactory(new int[] { 3 }, (INormalizedKeyComputerFactory) null,
+        MicroSortRuntimeFactory sort =
+                new MicroSortRuntimeFactory(new int[] { 3 }, (INormalizedKeyComputerFactory) null,
                         new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, null, 50);
 
         // the group-by
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
index 6f5587f..8c81dba 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
@@ -101,6 +101,12 @@
 
     @Override
     public void reset() {
+        // TODO(ali): fix to not release resources
+        heap.reset();
+    }
+
+    @Override
+    public void close() {
         heap.reset();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
index 819ff80..881db15 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
@@ -38,15 +38,17 @@
         }
     }
 
+    private final int initialNumFrames;
     private FrameSpace[] frameSpaces;
     private int size;
 
-    public FrameFreeSlotLastFit(int initialFrameNumber) {
-        frameSpaces = new FrameSpace[initialFrameNumber];
+    FrameFreeSlotLastFit(int initialFrameNumber) {
+        initialNumFrames = initialFrameNumber;
+        frameSpaces = new FrameSpace[initialNumFrames];
         size = 0;
     }
 
-    public FrameFreeSlotLastFit() {
+    FrameFreeSlotLastFit() {
         this(INITIAL_CAPACITY);
     }
 
@@ -65,6 +67,9 @@
 
     @Override
     public void pushNewFrame(int frameID, int freeSpace) {
+        if (frameSpaces == null) {
+            frameSpaces = new FrameSpace[initialNumFrames];
+        }
         if (size >= frameSpaces.length) {
             frameSpaces = Arrays.copyOf(frameSpaces, size * 2);
         }
@@ -78,8 +83,14 @@
     @Override
     public void reset() {
         size = 0;
+    }
+
+    @Override
+    public void close() {
+        size = 0;
         for (int i = frameSpaces.length - 1; i >= 0; i--) {
             frameSpaces[i] = null;
         }
+        frameSpaces = null;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
index ada6752..77518ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
@@ -27,7 +27,7 @@
 
     private TreeMap<Integer, LinkedList<Integer>> freeSpaceIndex;
 
-    public FrameFreeSlotSmallestFit() {
+    FrameFreeSlotSmallestFit() {
         freeSpaceIndex = new TreeMap<>();
     }
 
@@ -58,6 +58,12 @@
 
     @Override
     public void reset() {
+        // TODO(ali): fix to not release resources
+        freeSpaceIndex.clear();
+    }
+
+    @Override
+    public void close() {
         freeSpaceIndex.clear();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
index 1118bf3..8b73b54 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
@@ -23,19 +23,19 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Manage the buffer space in the unit of frame.
+ * Manages the buffer space in the unit of frame.
  */
 public interface IFrameBufferManager {
 
     /**
-     * Reset the counters and flags to initial status. This method should not release the pre-allocated resources
+     * Resets the counters and flags to initial status. This method should not release the pre-allocated resources
      *
      * @throws org.apache.hyracks.api.exceptions.HyracksDataException
      */
     void reset() throws HyracksDataException;
 
     /**
-     * @param frameIndex
+     * @param frameIndex index of frame requested
      * @param bufferInfo the given object need to be reset
      * @return the filled bufferInfo to facilitate the chain access
      */
@@ -54,6 +54,9 @@
      */
     int insertFrame(ByteBuffer frame) throws HyracksDataException;
 
+    /**
+     * Releases the allocated resources.
+     */
     void close();
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
index 8a1e004..5679844 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
@@ -22,8 +22,8 @@
 public interface IFrameFreeSlotPolicy {
 
     /**
-     * Find the best fit frame id which can hold the data, and then pop it out from the index.
-     * Return -1 is failed to find any.
+     * Finds the best fit frame id which can hold the data, and then pops it out from the index.
+     * Returns -1 if failed to find any.
      *
      * @param tobeInsertedSize the actual size of the data which should include
      *                         the meta data like the field offset and the tuple
@@ -33,16 +33,20 @@
     int popBestFit(int tobeInsertedSize);
 
     /**
-     * Register the new free slot into the index
+     * Registers the new free slot into the index.
      *
-     * @param frameID
-     * @param freeSpace
+     * @param frameID frame id
+     * @param freeSpace how much free space exists in this frame
      */
     void pushNewFrame(int frameID, int freeSpace);
 
     /**
-     * Clear all the existing free slot information.
+     * Clears all the existing free slot information. This method should not release the allocated resources.
      */
     void reset();
 
+    /**
+     * Releases the allocated resources.
+     */
+    void close();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
index 6c67ecc..33d0fda 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
@@ -158,7 +158,7 @@
     @Override
     public void close() {
         pool.close();
-        policy.reset();
+        policy.close();
         frames.clear();
         numTuples = 0;
         if (LOG.isDebugEnabled()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
index 6604ba8..50db2ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
@@ -37,18 +37,23 @@
             physicalFrame = frame;
             physicalOffset = offset;
         }
+
+        void reset(ByteBuffer frame, int offset) {
+            physicalFrame = frame;
+            physicalOffset = offset;
+        }
     }
 
     private final IFramePool framePool;
-    private List<PhysicalFrameOffset> physicalFrameOffsets;
-    private List<BufferInfo> logicalFrameStartSizes;
     private final IFrameFreeSlotPolicy freeSlotPolicy;
+    private final List<PhysicalFrameOffset> physicalFrames = new ArrayList<>();
+    private final List<BufferInfo> logicalFrames = new ArrayList<>();
+    private int numPhysicalFrames = 0;
+    private int numLogicalFrames = 0;
 
     public VariableFrameMemoryManager(IFramePool framePool, IFrameFreeSlotPolicy freeSlotPolicy) {
         this.framePool = framePool;
         this.freeSlotPolicy = freeSlotPolicy;
-        this.physicalFrameOffsets = new ArrayList<>();
-        this.logicalFrameStartSizes = new ArrayList<>();
     }
 
     private int findAvailableFrame(int frameSize) throws HyracksDataException {
@@ -59,29 +64,37 @@
         ByteBuffer buffer = framePool.allocateFrame(frameSize);
         if (buffer != null) {
             IntSerDeUtils.putInt(buffer.array(), FrameHelper.getTupleCountOffset(buffer.capacity()), 0);
-            physicalFrameOffsets.add(new PhysicalFrameOffset(buffer, 0));
-            return physicalFrameOffsets.size() - 1;
+            if (numPhysicalFrames < physicalFrames.size()) {
+                physicalFrames.get(numPhysicalFrames).reset(buffer, 0);
+            } else {
+                physicalFrames.add(new PhysicalFrameOffset(buffer, 0));
+            }
+            numPhysicalFrames++;
+            return numPhysicalFrames - 1; // returns the index of the physical frame appended
         }
         return -1;
     }
 
     @Override
     public void reset() throws HyracksDataException {
-        physicalFrameOffsets.clear();
-        logicalFrameStartSizes.clear();
+        numPhysicalFrames = 0;
+        numLogicalFrames = 0;
         freeSlotPolicy.reset();
         framePool.reset();
     }
 
     @Override
     public BufferInfo getFrame(int frameIndex, BufferInfo info) {
-        info.reset(logicalFrameStartSizes.get(frameIndex));
+        if (frameIndex >= numLogicalFrames) {
+            throw new IndexOutOfBoundsException();
+        }
+        info.reset(logicalFrames.get(frameIndex));
         return info;
     }
 
     @Override
     public int getNumFrames() {
-        return logicalFrameStartSizes.size();
+        return numLogicalFrames;
     }
 
     @Override
@@ -91,7 +104,7 @@
         if (physicalFrameId < 0) {
             return -1;
         }
-        PhysicalFrameOffset frameOffset = physicalFrameOffsets.get(physicalFrameId);
+        PhysicalFrameOffset frameOffset = physicalFrames.get(physicalFrameId);
         ByteBuffer buffer = frameOffset.physicalFrame;
         int offset = frameOffset.physicalOffset;
         System.arraycopy(frame.array(), 0, buffer.array(), offset, frameSize);
@@ -99,15 +112,22 @@
             freeSlotPolicy.pushNewFrame(physicalFrameId, buffer.capacity() - offset - frameSize);
         }
         frameOffset.physicalOffset = offset + frameSize;
-        logicalFrameStartSizes.add(new BufferInfo(buffer, offset, frameSize));
-        return logicalFrameStartSizes.size() - 1;
+        if (numLogicalFrames < logicalFrames.size()) {
+            logicalFrames.get(numLogicalFrames).reset(buffer, offset, frameSize);
+        } else {
+            logicalFrames.add(new BufferInfo(buffer, offset, frameSize));
+        }
+        numLogicalFrames++;
+        return numLogicalFrames - 1; // returns the index of the logical frame appended
     }
 
     @Override
     public void close() {
-        physicalFrameOffsets.clear();
-        logicalFrameStartSizes.clear();
-        freeSlotPolicy.reset();
+        numPhysicalFrames = 0;
+        numLogicalFrames = 0;
+        physicalFrames.clear();
+        logicalFrames.clear();
+        freeSlotPolicy.close();
         framePool.close();
     }
 }

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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:25:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/hKfzzKdFUjkU8Df86 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:30:33 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/975/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:38 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/880/ (8/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:05 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 23:42:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/643/ (12/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:12 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:06:16 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:27:40 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11191/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:17 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:08:16 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/6071/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:40 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6281/ (2/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:38 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/bgvjuXkUHCmtKbfQ9 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:10:00 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:04:59 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org>.
Ali Alsuliman has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3372/2/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
File hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java:

https://asterix-gerrit.ics.uci.edu/#/c/3372/2/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java@104
PS2, Line 104: // TODO(ali): fix to not release resources
> In this change?
If you insist. But I would prefer to do it in a later change to keep this one small and manageable.



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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Fri, 03 May 2019 17:57:10 +0000
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:04:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:41 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5698/ (6/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:40 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5701/ (11/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:10 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/6074/ (9/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:06 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:04:58 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/XjBJ9iKnWgSGsy9m7 : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 23:55:59 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:39 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:46:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9657/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:53 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/978/ (5/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:04:59 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 04:19:25 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:01 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Dmitry Lychagin (Code Review)" <de...@asterixdb.apache.org>.
Dmitry Lychagin has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2: Code-Review+1


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Fri, 03 May 2019 17:30:39 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2: Contrib+1

Analytics Compatibility Tests Successful
https://cbjenkins.page.link/2gJkSm3sDJBH6c2y6 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 05:09:29 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Till Westmann (Code Review)" <de...@asterixdb.apache.org>.
Till Westmann has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2: Code-Review+2


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Sat, 04 May 2019 21:18:18 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:18 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5717/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:15 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/482/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:13 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:30:01 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5677/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:51 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/640/ (10/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:45 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5614/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:37 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:49 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5714/ (9/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:43 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/5921/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:55 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1070/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:05:03 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/479/ (11/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5617/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 02 May 2019 03:04:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org>.
Hello Anon. E. Moose (1000171), Jenkins, 

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

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

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

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................

[ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

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

Details:
The "VariableFrameMemoryManager" releases the physical frames
objects, the logical frames objects and the free slots objects
on reset(). This patch is to retain those objects and only
release them on close() (as also stated in the interface).

- added close() to IFrameFreeSlotPolicy to make the interface similar
to the other interfaces (IFrameBufferManager, IFramePool, ... etc)
where close() is supposed to release while reset() is supposed to only
reset but retain resources.
- renamed "InMemorySortRuntimeFactory" to "MicroSortRuntimeFactory".

Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
---
M hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
R hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
M hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
10 files changed, 90 insertions(+), 43 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/72/3372/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3372
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Ali Alsuliman (Code Review)" <de...@asterixdb.apache.org>.
Ali Alsuliman has submitted this change and it was merged. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................

[ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

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

Details:
The "VariableFrameMemoryManager" releases the physical frames
objects, the logical frames objects and the free slots objects
on reset(). This patch is to retain those objects and only
release them on close() (as also stated in the interface).

- added close() to IFrameFreeSlotPolicy to make the interface similar
to the other interfaces (IFrameBufferManager, IFramePool, ... etc)
where close() is supposed to release while reset() is supposed to only
reset but retain resources.
- renamed "InMemorySortRuntimeFactory" to "MicroSortRuntimeFactory".

Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3372
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
Reviewed-by: Till Westmann <ti...@apache.org>
---
M hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
R hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
M hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
M hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
10 files changed, 90 insertions(+), 43 deletions(-)

Approvals:
  Jenkins: Verified; ; Verified
  Anon. E. Moose (1000171): 
  Dmitry Lychagin: Looks good to me, but someone else must approve
  Till Westmann: Looks good to me, approved

Objections:
  Jenkins: Violations found



diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
index c6a7d9d..413c1a4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MicroStableSortPOperator.java
@@ -32,7 +32,7 @@
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.sort.InMemorySortRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.sort.MicroSortRuntimeFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -80,7 +80,7 @@
             i++;
         }
 
-        IPushRuntimeFactory runtime = new InMemorySortRuntimeFactory(sortFields, nkcf, comps, null, maxNumberOfFrames);
+        IPushRuntimeFactory runtime = new MicroSortRuntimeFactory(sortFields, nkcf, comps, null, maxNumberOfFrames);
         builder.contributeMicroOperator(op, runtime, recDescriptor);
         ILogicalOperator src = op.getInputs().get(0).getValue();
         builder.contributeGraphEdge(src, 0, op, 0);
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
similarity index 96%
rename from hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
rename to hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
index b34b7b9..ad44c90 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/InMemorySortRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/sort/MicroSortRuntimeFactory.java
@@ -39,7 +39,7 @@
 import org.apache.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
 import org.apache.hyracks.dataflow.std.sort.ExternalSortRunMerger;
 
-public class InMemorySortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
+public class MicroSortRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
     private final int framesLimit;
@@ -47,14 +47,14 @@
     private final INormalizedKeyComputerFactory[] keyNormalizerFactories;
     private final IBinaryComparatorFactory[] comparatorFactories;
 
-    public InMemorySortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+    public MicroSortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
             IBinaryComparatorFactory[] comparatorFactories, int[] projectionList, int framesLimit) {
         this(sortFields, firstKeyNormalizerFactory != null
                 ? new INormalizedKeyComputerFactory[] { firstKeyNormalizerFactory } : null, comparatorFactories,
                 projectionList, framesLimit);
     }
 
-    public InMemorySortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory[] keyNormalizerFactories,
+    public MicroSortRuntimeFactory(int[] sortFields, INormalizedKeyComputerFactory[] keyNormalizerFactories,
             IBinaryComparatorFactory[] comparatorFactories, int[] projectionList, int framesLimit) {
         super(projectionList);
         // Obs: the projection list is currently ignored.
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
index e9b3fc3..aeb22b6 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-tests/src/test/java/org/apache/hyracks/algebricks/tests/pushruntime/PushRuntimeTest.java
@@ -47,7 +47,7 @@
 import org.apache.hyracks.algebricks.runtime.operators.group.MicroPreClusteredGroupRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
 import org.apache.hyracks.algebricks.runtime.operators.meta.SubplanRuntimeFactory;
-import org.apache.hyracks.algebricks.runtime.operators.sort.InMemorySortRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.sort.MicroSortRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
 import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory;
@@ -718,8 +718,8 @@
                 new String[] { AlgebricksHyracksIntegrationUtil.NC1_ID });
 
         // the algebricks op.
-        InMemorySortRuntimeFactory sort =
-                new InMemorySortRuntimeFactory(new int[] { 1 }, (INormalizedKeyComputerFactory) null,
+        MicroSortRuntimeFactory sort =
+                new MicroSortRuntimeFactory(new int[] { 1 }, (INormalizedKeyComputerFactory) null,
                         new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }, null, 50);
         RecordDescriptor sortDesc = scannerDesc;
 
@@ -834,8 +834,8 @@
 
         // the sort (by nation id)
         RecordDescriptor sortDesc = scannerDesc;
-        InMemorySortRuntimeFactory sort =
-                new InMemorySortRuntimeFactory(new int[] { 3 }, (INormalizedKeyComputerFactory) null,
+        MicroSortRuntimeFactory sort =
+                new MicroSortRuntimeFactory(new int[] { 3 }, (INormalizedKeyComputerFactory) null,
                         new IBinaryComparatorFactory[] { IntegerBinaryComparatorFactory.INSTANCE }, null, 50);
 
         // the group-by
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
index 6f5587f..8c81dba 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
@@ -101,6 +101,12 @@
 
     @Override
     public void reset() {
+        // TODO(ali): fix to not release resources
+        heap.reset();
+    }
+
+    @Override
+    public void close() {
         heap.reset();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
index 819ff80..9afe6d1 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotLastFit.java
@@ -38,15 +38,17 @@
         }
     }
 
+    private final int initialNumFrames;
     private FrameSpace[] frameSpaces;
     private int size;
 
-    public FrameFreeSlotLastFit(int initialFrameNumber) {
-        frameSpaces = new FrameSpace[initialFrameNumber];
+    FrameFreeSlotLastFit(int initialFrameNumber) {
+        initialNumFrames = initialFrameNumber;
+        frameSpaces = new FrameSpace[initialNumFrames];
         size = 0;
     }
 
-    public FrameFreeSlotLastFit() {
+    FrameFreeSlotLastFit() {
         this(INITIAL_CAPACITY);
     }
 
@@ -65,6 +67,9 @@
 
     @Override
     public void pushNewFrame(int frameID, int freeSpace) {
+        if (frameSpaces == null) {
+            frameSpaces = new FrameSpace[initialNumFrames];
+        }
         if (size >= frameSpaces.length) {
             frameSpaces = Arrays.copyOf(frameSpaces, size * 2);
         }
@@ -78,8 +83,11 @@
     @Override
     public void reset() {
         size = 0;
-        for (int i = frameSpaces.length - 1; i >= 0; i--) {
-            frameSpaces[i] = null;
-        }
+    }
+
+    @Override
+    public void close() {
+        size = 0;
+        frameSpaces = null;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
index ada6752..77518ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotSmallestFit.java
@@ -27,7 +27,7 @@
 
     private TreeMap<Integer, LinkedList<Integer>> freeSpaceIndex;
 
-    public FrameFreeSlotSmallestFit() {
+    FrameFreeSlotSmallestFit() {
         freeSpaceIndex = new TreeMap<>();
     }
 
@@ -58,6 +58,12 @@
 
     @Override
     public void reset() {
+        // TODO(ali): fix to not release resources
+        freeSpaceIndex.clear();
+    }
+
+    @Override
+    public void close() {
         freeSpaceIndex.clear();
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
index 1118bf3..8b73b54 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
@@ -23,19 +23,19 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 /**
- * Manage the buffer space in the unit of frame.
+ * Manages the buffer space in the unit of frame.
  */
 public interface IFrameBufferManager {
 
     /**
-     * Reset the counters and flags to initial status. This method should not release the pre-allocated resources
+     * Resets the counters and flags to initial status. This method should not release the pre-allocated resources
      *
      * @throws org.apache.hyracks.api.exceptions.HyracksDataException
      */
     void reset() throws HyracksDataException;
 
     /**
-     * @param frameIndex
+     * @param frameIndex index of frame requested
      * @param bufferInfo the given object need to be reset
      * @return the filled bufferInfo to facilitate the chain access
      */
@@ -54,6 +54,9 @@
      */
     int insertFrame(ByteBuffer frame) throws HyracksDataException;
 
+    /**
+     * Releases the allocated resources.
+     */
     void close();
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
index 8a1e004..5679844 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameFreeSlotPolicy.java
@@ -22,8 +22,8 @@
 public interface IFrameFreeSlotPolicy {
 
     /**
-     * Find the best fit frame id which can hold the data, and then pop it out from the index.
-     * Return -1 is failed to find any.
+     * Finds the best fit frame id which can hold the data, and then pops it out from the index.
+     * Returns -1 if failed to find any.
      *
      * @param tobeInsertedSize the actual size of the data which should include
      *                         the meta data like the field offset and the tuple
@@ -33,16 +33,20 @@
     int popBestFit(int tobeInsertedSize);
 
     /**
-     * Register the new free slot into the index
+     * Registers the new free slot into the index.
      *
-     * @param frameID
-     * @param freeSpace
+     * @param frameID frame id
+     * @param freeSpace how much free space exists in this frame
      */
     void pushNewFrame(int frameID, int freeSpace);
 
     /**
-     * Clear all the existing free slot information.
+     * Clears all the existing free slot information. This method should not release the allocated resources.
      */
     void reset();
 
+    /**
+     * Releases the allocated resources.
+     */
+    void close();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
index 6c67ecc..33d0fda 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
@@ -158,7 +158,7 @@
     @Override
     public void close() {
         pool.close();
-        policy.reset();
+        policy.close();
         frames.clear();
         numTuples = 0;
         if (LOG.isDebugEnabled()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
index 6604ba8..50db2ec 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
@@ -37,18 +37,23 @@
             physicalFrame = frame;
             physicalOffset = offset;
         }
+
+        void reset(ByteBuffer frame, int offset) {
+            physicalFrame = frame;
+            physicalOffset = offset;
+        }
     }
 
     private final IFramePool framePool;
-    private List<PhysicalFrameOffset> physicalFrameOffsets;
-    private List<BufferInfo> logicalFrameStartSizes;
     private final IFrameFreeSlotPolicy freeSlotPolicy;
+    private final List<PhysicalFrameOffset> physicalFrames = new ArrayList<>();
+    private final List<BufferInfo> logicalFrames = new ArrayList<>();
+    private int numPhysicalFrames = 0;
+    private int numLogicalFrames = 0;
 
     public VariableFrameMemoryManager(IFramePool framePool, IFrameFreeSlotPolicy freeSlotPolicy) {
         this.framePool = framePool;
         this.freeSlotPolicy = freeSlotPolicy;
-        this.physicalFrameOffsets = new ArrayList<>();
-        this.logicalFrameStartSizes = new ArrayList<>();
     }
 
     private int findAvailableFrame(int frameSize) throws HyracksDataException {
@@ -59,29 +64,37 @@
         ByteBuffer buffer = framePool.allocateFrame(frameSize);
         if (buffer != null) {
             IntSerDeUtils.putInt(buffer.array(), FrameHelper.getTupleCountOffset(buffer.capacity()), 0);
-            physicalFrameOffsets.add(new PhysicalFrameOffset(buffer, 0));
-            return physicalFrameOffsets.size() - 1;
+            if (numPhysicalFrames < physicalFrames.size()) {
+                physicalFrames.get(numPhysicalFrames).reset(buffer, 0);
+            } else {
+                physicalFrames.add(new PhysicalFrameOffset(buffer, 0));
+            }
+            numPhysicalFrames++;
+            return numPhysicalFrames - 1; // returns the index of the physical frame appended
         }
         return -1;
     }
 
     @Override
     public void reset() throws HyracksDataException {
-        physicalFrameOffsets.clear();
-        logicalFrameStartSizes.clear();
+        numPhysicalFrames = 0;
+        numLogicalFrames = 0;
         freeSlotPolicy.reset();
         framePool.reset();
     }
 
     @Override
     public BufferInfo getFrame(int frameIndex, BufferInfo info) {
-        info.reset(logicalFrameStartSizes.get(frameIndex));
+        if (frameIndex >= numLogicalFrames) {
+            throw new IndexOutOfBoundsException();
+        }
+        info.reset(logicalFrames.get(frameIndex));
         return info;
     }
 
     @Override
     public int getNumFrames() {
-        return logicalFrameStartSizes.size();
+        return numLogicalFrames;
     }
 
     @Override
@@ -91,7 +104,7 @@
         if (physicalFrameId < 0) {
             return -1;
         }
-        PhysicalFrameOffset frameOffset = physicalFrameOffsets.get(physicalFrameId);
+        PhysicalFrameOffset frameOffset = physicalFrames.get(physicalFrameId);
         ByteBuffer buffer = frameOffset.physicalFrame;
         int offset = frameOffset.physicalOffset;
         System.arraycopy(frame.array(), 0, buffer.array(), offset, frameSize);
@@ -99,15 +112,22 @@
             freeSlotPolicy.pushNewFrame(physicalFrameId, buffer.capacity() - offset - frameSize);
         }
         frameOffset.physicalOffset = offset + frameSize;
-        logicalFrameStartSizes.add(new BufferInfo(buffer, offset, frameSize));
-        return logicalFrameStartSizes.size() - 1;
+        if (numLogicalFrames < logicalFrames.size()) {
+            logicalFrames.get(numLogicalFrames).reset(buffer, offset, frameSize);
+        } else {
+            logicalFrames.add(new BufferInfo(buffer, offset, frameSize));
+        }
+        numLogicalFrames++;
+        return numLogicalFrames - 1; // returns the index of the logical frame appended
     }
 
     @Override
     public void close() {
-        physicalFrameOffsets.clear();
-        logicalFrameStartSizes.clear();
-        freeSlotPolicy.reset();
+        numPhysicalFrames = 0;
+        numLogicalFrames = 0;
+        physicalFrames.clear();
+        logicalFrames.clear();
+        freeSlotPolicy.close();
         framePool.close();
     }
 }

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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1068/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Till Westmann (Code Review)" <de...@asterixdb.apache.org>.
Till Westmann has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 2:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3372/2/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java
File hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java:

https://asterix-gerrit.ics.uci.edu/#/c/3372/2/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/FrameFreeSlotBiggestFirst.java@104
PS2, Line 104: // TODO(ali): fix to not release resources
In this change?



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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Fri, 03 May 2019 15:50:31 +0000
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3372 )

Change subject: [ASTERIXDB-2552][RT] Reuse objects on reseting buffer manager
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/877/ (4/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I88bea27e5024c621412ef609475e2a7ba1913afa
Gerrit-Change-Number: 3372
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 01 May 2019 22:26:39 +0000
Gerrit-HasComments: No