You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by pr...@apache.org on 2016/10/17 19:55:29 UTC

[39/50] [abbrv] asterixdb git commit: reuse pairs for frame tuple appender.

reuse pairs for frame tuple appender.


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/f6dba461
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/f6dba461
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/f6dba461

Branch: refs/heads/ecarm002/interval_join_merge
Commit: f6dba4610812497aa595b7d27c44458b71f798ad
Parents: 1ecdb32
Author: Preston Carman <pr...@apache.org>
Authored: Sun Sep 25 15:08:57 2016 -0700
Committer: Preston Carman <pr...@apache.org>
Committed: Sun Sep 25 15:08:57 2016 -0700

----------------------------------------------------------------------
 .../sort/util/DeletableFrameTupleAppender.java  | 15 +++-
 .../dataflow/std/sort/util/IntegerPair.java     | 87 ++++++++++++++++++++
 .../dataflow/std/sort/util/IntegerPairPool.java | 39 +++++++++
 3 files changed, 137 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/f6dba461/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
index e5f0bce2..c143eaf 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
@@ -25,6 +25,7 @@ import java.util.Comparator;
 import java.util.PriorityQueue;
 
 import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.MutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -102,12 +103,13 @@ public class DeletableFrameTupleAppender implements IAppendDeletableFrameTupleAc
     private int deletedSpace;
     private int nextIndex;
     private byte[] array; // to speed up the array visit a little
+    private IntegerPairPool ipp = new IntegerPairPool();
 
-    private final PriorityQueue<Pair<Integer, Integer>> reorganizeQueue;
+    private final PriorityQueue<IntegerPair> reorganizeQueue;
 
     public DeletableFrameTupleAppender(RecordDescriptor recordDescriptor) {
         this.recordDescriptor = recordDescriptor;
-        reorganizeQueue = new PriorityQueue<>(16, INDEX_OFFSET_ASC_COMPARATOR);
+        reorganizeQueue = new PriorityQueue<>(16, IntegerPair.RIGHT_ASC_COMPARATOR);
     }
 
     private int getIndexCount() {
@@ -251,19 +253,23 @@ public class DeletableFrameTupleAppender implements IAppendDeletableFrameTupleAc
         reclaimDeletedEnding();
 
         // Build reorganize queue
+        IntegerPair ip;
         int endOffset;
         int startOffset;
         for (int i = 0; i < indexCount; i++) {
             endOffset = getTupleEndOffset(i);
             if (endOffset > 0) {
-                reorganizeQueue.add(new ImmutablePair<Integer, Integer>(i, getTupleStartOffset(i)));
+                ip = ipp.takeOne();
+                ip.reset(i, getTupleStartOffset(i));
+                reorganizeQueue.add(ip);
             }
         }
 
         int index;
         tupleAppend = 0;
         while (!reorganizeQueue.isEmpty()) {
-            index = reorganizeQueue.remove().getKey();
+            ip = reorganizeQueue.remove();
+            index = ip.getLeft();
             startOffset = getTupleStartOffset(index);
             endOffset = getTupleEndOffset(index);
             if (endOffset >= 0) {
@@ -275,6 +281,7 @@ public class DeletableFrameTupleAppender implements IAppendDeletableFrameTupleAc
                 setTupleOffsets(index, tupleAppend, length);
                 tupleAppend += length;
             }
+            ipp.giveBack(ip);
         }
         setTupleAppend(tupleAppend);
         deletedSpace = 0;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/f6dba461/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPair.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPair.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPair.java
new file mode 100644
index 0000000..99f327b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPair.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.sort.util;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+public class IntegerPair implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public static final Comparator<IntegerPair> LEFT_ASC_COMPARATOR = new Comparator<IntegerPair>() {
+        @Override
+        public int compare(IntegerPair p1, IntegerPair p2) {
+            return p1.getLeft() - p2.getLeft();
+        }
+
+    };
+
+    public static final Comparator<IntegerPair> RIGHT_ASC_COMPARATOR = new Comparator<IntegerPair>() {
+        @Override
+        public int compare(IntegerPair p1, IntegerPair p2) {
+            return p1.getRight() - p2.getRight();
+        }
+
+    };
+
+    private int left;
+    private int right;
+
+    public IntegerPair() {
+        reset(Integer.MIN_VALUE, Integer.MIN_VALUE);
+    }
+
+    public IntegerPair(int l, int r) {
+        reset(l, r);
+    }
+
+    public int getLeft() {
+        return left;
+    }
+
+    public int getRight() {
+        return right;
+    }
+
+    public void reset(int l, int r) {
+        left = l;
+        right = r;
+    }
+
+    @Override
+    public String toString() {
+        return left + "," + right;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof IntegerPair)) {
+            return false;
+        } else {
+            IntegerPair p = (IntegerPair) obj;
+            return this.left == p.getLeft() && this.right == p.getRight();
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return left * 31 + right;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/f6dba461/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPairPool.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPairPool.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPairPool.java
new file mode 100644
index 0000000..330e822
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPairPool.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hyracks.dataflow.std.sort.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class IntegerPairPool {
+    private final List<IntegerPair> list;
+
+    public IntegerPairPool() {
+        list = new ArrayList<>();
+    }
+
+    public IntegerPair takeOne() {
+        if (list.isEmpty()) {
+            return new IntegerPair();
+        }
+        return list.remove(list.size() - 1);
+    }
+
+    public void giveBack(IntegerPair pair) {
+        list.add(pair);
+    }
+}