Preston Carman has uploaded a new change for review.

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

Change subject: Sonar fixes and other clean up items.
......................................................................

Sonar fixes and other clean up items.

Change-Id: Ieb72a0fef3c181282cc4c71b881a7cb52da86818
---
M 
asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
M 
asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
M 
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
M 
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
M 
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
M 
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
M 
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
M 
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
M 
asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
A 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractExpressionAnnotation.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
M 
hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
M 
hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractStateObject.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
R 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
M 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java
M 
hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M 
hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M 
hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M 
hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M 
hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M 
hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M 
hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
M 
hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
M 
hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
54 files changed, 278 insertions(+), 260 deletions(-)


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

diff --git 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
index 0daa551..de1e1fa 100644
--- 
a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
+++ 
b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
@@ -18,24 +18,13 @@
  */
 package org.apache.asterix.common.annotations;
 
+import 
org.apache.hyracks.algebricks.core.algebra.expressions.AbstractExpressionAnnotation;
 import 
org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 
-public class SkipSecondaryIndexSearchExpressionAnnotation implements 
IExpressionAnnotation {
+public class SkipSecondaryIndexSearchExpressionAnnotation extends 
AbstractExpressionAnnotation {
 
-    public static final String SKIP_SECONDARY_INDEX_SEARCH_ANNOTATION_KEY = 
"skip-index";
+    public static final String HINT_STRING = "skip-index";
     public static final SkipSecondaryIndexSearchExpressionAnnotation INSTANCE 
= new SkipSecondaryIndexSearchExpressionAnnotation();
-
-    private Object object;
-
-    @Override
-    public Object getObject() {
-        return object;
-    }
-
-    @Override
-    public void setObject(Object object) {
-        this.object = object;
-    }
 
     @Override
     public IExpressionAnnotation copy() {
@@ -46,6 +35,6 @@
 
     @Override
     public String toString() {
-        return SKIP_SECONDARY_INDEX_SEARCH_ANNOTATION_KEY;
+        return HINT_STRING;
     }
 }
diff --git 
a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
 
b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
index 5b333e5..5430393 100644
--- 
a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
+++ 
b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
@@ -25,7 +25,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 
 public class AIntervalTypeComputer extends AbstractResultTypeComputer {
-
+    private static final long serialVersionUID = 1L;
     public static final AIntervalTypeComputer INSTANCE = new 
AIntervalTypeComputer();
 
     private AIntervalTypeComputer() {
diff --git 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
index f2be58f..bcfa333 100644
--- 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
+++ 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
@@ -20,6 +20,7 @@
 
 import java.io.Serializable;
 
+import org.apache.asterix.common.exceptions.AsterixException;
 import 
org.apache.asterix.dataflow.data.nontagged.comparators.ACirclePartialBinaryComparatorFactory;
 import 
org.apache.asterix.dataflow.data.nontagged.comparators.ADurationPartialBinaryComparatorFactory;
 import 
org.apache.asterix.dataflow.data.nontagged.comparators.AIntervalAscPartialBinaryComparatorFactory;
@@ -37,7 +38,6 @@
 import 
org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -73,7 +73,7 @@
             ByteArrayPointable.FACTORY).createBinaryComparator();
 
     public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, 
IPointable arg2)
-            throws AlgebricksException {
+            throws AsterixException {
         switch (typeTag1) {
             case INT8:
                 return compareInt8WithArg(typeTag2, arg1, arg2);
@@ -97,9 +97,9 @@
     }
 
     private int compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag 
actualTypeTag, IPointable arg1,
-            IPointable arg2) throws AlgebricksException {
+            IPointable arg2) throws AsterixException {
         if (expectedTypeTag != actualTypeTag) {
-            throw new AlgebricksException(
+            throw new AsterixException(
                     "Comparison is undefined between " + expectedTypeTag + " 
and " + actualTypeTag + ".");
         }
         int result = 0;
@@ -164,38 +164,38 @@
                             rightLen);
                     break;
                 default:
-                    throw new AlgebricksException("Comparison for " + 
actualTypeTag + " is not supported.");
+                    throw new AsterixException("Comparison for " + 
actualTypeTag + " is not supported.");
             }
         } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
+            throw new AsterixException(e);
         }
         return result;
     }
 
-    private int compareBooleanWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareBooleanWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         if (typeTag2 == ATypeTag.BOOLEAN) {
             byte b0 = arg1.getByteArray()[arg1.getStartOffset()];
             byte b1 = arg2.getByteArray()[arg2.getStartOffset()];
             return compareByte(b0, b1);
         }
-        throw new AlgebricksException("Comparison is undefined between types 
ABoolean and " + typeTag2 + " .");
+        throw new AsterixException("Comparison is undefined between types 
ABoolean and " + typeTag2 + " .");
     }
 
-    private int compareStringWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareStringWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         if (typeTag2 == ATypeTag.STRING) {
             int result;
             try {
                 result = strBinaryComp.compare(arg1.getByteArray(), 
arg1.getStartOffset(), arg1.getLength() - 1,
                         arg2.getByteArray(), arg2.getStartOffset(), 
arg2.getLength() - 1);
             } catch (HyracksDataException e) {
-                throw new AlgebricksException(e);
+                throw new AsterixException(e);
             }
             return result;
         }
-        throw new AlgebricksException("Comparison is undefined between types 
AString and " + typeTag2 + " .");
+        throw new AsterixException("Comparison is undefined between types 
AString and " + typeTag2 + " .");
     }
 
-    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
@@ -228,12 +228,12 @@
                 return compareDouble(s, v2);
             }
             default: {
-                throw new AlgebricksException("Comparison is undefined between 
types ADouble and " + typeTag2 + " .");
+                throw new AsterixException("Comparison is undefined between 
types ADouble and " + typeTag2 + " .");
             }
         }
     }
 
-    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
@@ -266,12 +266,12 @@
                 return compareDouble(s, v2);
             }
             default: {
-                throw new AlgebricksException("Comparison is undefined between 
types AFloat and " + typeTag2 + " .");
+                throw new AsterixException("Comparison is undefined between 
types AFloat and " + typeTag2 + " .");
             }
         }
     }
 
-    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
@@ -304,12 +304,12 @@
                 return compareDouble(s, v2);
             }
             default: {
-                throw new AlgebricksException("Comparison is undefined between 
types AInt64 and " + typeTag2 + " .");
+                throw new AsterixException("Comparison is undefined between 
types AInt64 and " + typeTag2 + " .");
             }
         }
     }
 
-    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
@@ -342,12 +342,12 @@
                 return compareDouble(s, v2);
             }
             default: {
-                throw new AlgebricksException("Comparison is undefined between 
types AInt32 and " + typeTag2 + " .");
+                throw new AsterixException("Comparison is undefined between 
types AInt32 and " + typeTag2 + " .");
             }
         }
     }
 
-    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         byte[] leftBytes = arg1.getByteArray();
         int leftOffset = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
@@ -380,12 +380,12 @@
                 return compareDouble(s, v2);
             }
             default: {
-                throw new AlgebricksException("Comparison is undefined between 
types AInt16 and " + typeTag2 + " .");
+                throw new AsterixException("Comparison is undefined between 
types AInt16 and " + typeTag2 + " .");
             }
         }
     }
 
-    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AlgebricksException {
+    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, 
IPointable arg2) throws AsterixException {
         byte[] leftBytes = arg1.getByteArray();
         int leftStart = arg1.getStartOffset();
         byte[] rightBytes = arg2.getByteArray();
@@ -418,7 +418,7 @@
                 return compareDouble(s, v2);
             }
             default: {
-                throw new AlgebricksException("Comparison is undefined between 
types AInt16 and " + typeTag2 + " .");
+                throw new AsterixException("Comparison is undefined between 
types AInt16 and " + typeTag2 + " .");
             }
         }
     }
diff --git 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
index ccfdb43..58a65e4 100644
--- 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
+++ 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalConstructorDescriptor.java
@@ -125,8 +125,6 @@
                             intervalSerde.serialize(aInterval, out);
                         } catch (IOException e1) {
                             throw new AlgebricksException(errorMessage);
-                        } catch (Exception e2) {
-                            throw new AlgebricksException(e2);
                         }
                         result.set(resultStorage);
                     }
diff --git 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
index 1de1e48..39a0a36 100644
--- 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
+++ 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
@@ -108,8 +108,7 @@
                                             + interval0.getTypeTag() + " and 
interval of " + interval1.getTypeTag());
                                 }
 
-                                if (il.overlap(interval0, interval1) || 
il.overlappedBy(interval0, interval1)
-                                        || il.covers(interval0, interval1) || 
il.coveredBy(interval0, interval1)) {
+                                if (il.overlapping(interval0, interval1)) {
                                     long start = 
Math.max(interval0.getStartValue(), interval1.getStartValue());
                                     long end = 
Math.min(interval0.getEndValue(), interval1.getEndValue());
                                     aInterval.setValue(start, end, 
intervalType0);
diff --git 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
index c8c3486..505fe83 100644
--- 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
+++ 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
@@ -20,13 +20,13 @@
 
 import java.io.Serializable;
 
+import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
 import org.apache.asterix.runtime.evaluators.comparisons.ComparisonHelper;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 
-public class IntervalLogic implements Serializable{
+public class IntervalLogic implements Serializable {
 
     private static final long serialVersionUID = 1L;
     private final ComparisonHelper ch = new ComparisonHelper();
@@ -35,7 +35,7 @@
     private final IPointable s2 = VoidPointable.FACTORY.createPointable();
     private final IPointable e2 = VoidPointable.FACTORY.createPointable();
 
-    public boolean validateInterval(AIntervalPointable ip1) throws 
AlgebricksException {
+    public boolean validateInterval(AIntervalPointable ip1) throws 
AsterixException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
         return ch.compare(ip1.getTypeTag(), ip1.getTypeTag(), s1, e1) <= 0;
@@ -47,16 +47,16 @@
      * @param ip1
      * @param ip2
      * @return boolean
-     * @throws AlgebricksException
+     * @throws AsterixException
      * @see #after(AIntervalPointable, AIntervalPointable)
      */
-    public boolean before(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean before(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         ip1.getEnd(e1);
         ip2.getStart(s2);
         return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) < 0;
     }
 
-    public boolean after(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean after(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         return before(ip2, ip1);
     }
 
@@ -66,16 +66,16 @@
      * @param ip1
      * @param ip2
      * @return boolean
-     * @throws AlgebricksException
+     * @throws AsterixException
      * @see #metBy(AIntervalPointable, AIntervalPointable)
      */
-    public boolean meets(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean meets(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         ip1.getEnd(e1);
         ip2.getStart(s2);
         return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) == 0;
     }
 
-    public boolean metBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean metBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         return meets(ip2, ip1);
     }
 
@@ -85,10 +85,10 @@
      * @param ip1
      * @param ip2
      * @return boolean
-     * @throws AlgebricksException
+     * @throws AsterixException
      * @see #overlappedBy(AIntervalPointable, AIntervalPointable)
      */
-    public boolean overlaps(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean overlaps(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
         ip2.getStart(s2);
@@ -98,7 +98,7 @@
                 && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) < 0;
     }
 
-    public boolean overlappedBy(AIntervalPointable ip1, AIntervalPointable 
ip2) throws AlgebricksException {
+    public boolean overlappedBy(AIntervalPointable ip1, AIntervalPointable 
ip2) throws AsterixException {
         return overlaps(ip2, ip1);
     }
 
@@ -107,18 +107,16 @@
      *
      * @param ip1
      * @param ip2
-     * @throws AlgebricksException
+     * @throws AsterixException
      * @return boolean
      */
-    public boolean overlap(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean overlapping(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
         ip2.getStart(s2);
         ip2.getEnd(e2);
-        return (ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2) <= 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) > 0)
-                || (ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) >= 0
-                        && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, 
e2) < 0);
+        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, e2) < 0
+                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) > 0;
     }
 
     /**
@@ -127,10 +125,10 @@
      * @param ip1
      * @param ip2
      * @return boolean
-     * @throws AlgebricksException
+     * @throws AsterixException
      * @see #startedBy(AIntervalPointable, AIntervalPointable)
      */
-    public boolean starts(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean starts(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
         ip2.getStart(s2);
@@ -139,7 +137,7 @@
                 && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) <= 0;
     }
 
-    public boolean startedBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean startedBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         return starts(ip2, ip1);
     }
 
@@ -149,10 +147,10 @@
      * @param ip1
      * @param ip2
      * @return boolean
-     * @throws AlgebricksException
+     * @throws AsterixException
      * @see #coveredBy(AIntervalPointable, AIntervalPointable)
      */
-    public boolean covers(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean covers(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
         ip2.getStart(s2);
@@ -161,7 +159,7 @@
                 && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) >= 0;
     }
 
-    public boolean coveredBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean coveredBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         return covers(ip2, ip1);
     }
 
@@ -171,10 +169,10 @@
      * @param ip1
      * @param ip2
      * @return boolean
-     * @throws AlgebricksException
+     * @throws AsterixException
      * @see #endedBy(AIntervalPointable, AIntervalPointable)
      */
-    public boolean ends(AIntervalPointable ip1, AIntervalPointable ip2) throws 
AlgebricksException {
+    public boolean ends(AIntervalPointable ip1, AIntervalPointable ip2) throws 
AsterixException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
         ip2.getStart(s2);
@@ -183,11 +181,11 @@
                 && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) == 0;
     }
 
-    public boolean endedBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean endedBy(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         return ends(ip2, ip1);
     }
 
-    public boolean equals(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AlgebricksException {
+    public boolean equals(AIntervalPointable ip1, AIntervalPointable ip2) 
throws AsterixException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
         ip2.getStart(s2);
diff --git 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
index 3e0a4e5..f18c81a 100644
--- 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
+++ 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappedByDescriptor.java
@@ -27,9 +27,9 @@
 
 public class IntervalOverlappedByDescriptor extends 
AbstractIntervalLogicFuncDescriptor {
 
-    private final static long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-    public final static IFunctionDescriptorFactory FACTORY = new 
IFunctionDescriptorFactory() {
+    public static final IFunctionDescriptorFactory FACTORY = new 
IFunctionDescriptorFactory() {
 
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
diff --git 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
index de5bd53..5e2a26e 100644
--- 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
+++ 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlapsDescriptor.java
@@ -27,9 +27,9 @@
 
 public class IntervalOverlapsDescriptor extends 
AbstractIntervalLogicFuncDescriptor {
 
-    private final static long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-    public final static IFunctionDescriptorFactory FACTORY = new 
IFunctionDescriptorFactory() {
+    public static final IFunctionDescriptorFactory FACTORY = new 
IFunctionDescriptorFactory() {
 
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
diff --git 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
index 7d3fe3f..2def89d 100644
--- 
a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
+++ 
b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
@@ -27,10 +27,9 @@
 
 public class OverlapDescriptor extends AbstractIntervalLogicFuncDescriptor {
 
-    private final static long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-    public final static IFunctionDescriptorFactory FACTORY = new 
IFunctionDescriptorFactory() {
-
+    public static final IFunctionDescriptorFactory FACTORY = new 
IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new OverlapDescriptor();
@@ -45,7 +44,7 @@
     @Override
     protected boolean compareIntervals(IntervalLogic il, AIntervalPointable 
ip1, AIntervalPointable ip2)
             throws AlgebricksException {
-        return il.overlap(ip1, ip2);
+        return il.overlapping(ip1, ip2);
     }
 
 }
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractExpressionAnnotation.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractExpressionAnnotation.java
new file mode 100644
index 0000000..de02572
--- /dev/null
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractExpressionAnnotation.java
@@ -0,0 +1,35 @@
+/*
+ * 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.algebricks.core.algebra.expressions;
+
+public abstract class AbstractExpressionAnnotation implements 
IExpressionAnnotation {
+
+    protected Object object;
+
+    @Override
+    public Object getObject() {
+        return object;
+    }
+
+    @Override
+    public void setObject(Object object) {
+        this.object = object;
+    }
+
+}
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java
index 140dfb1..3aa34c8 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/ExpressionAnnotationNoCopyImpl.java
@@ -18,23 +18,11 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.expressions;
 
-public class ExpressionAnnotationNoCopyImpl implements IExpressionAnnotation {
-
-    private Object object;
+public class ExpressionAnnotationNoCopyImpl extends 
AbstractExpressionAnnotation {
 
     @Override
     public IExpressionAnnotation copy() {
         return this;
-    }
-
-    @Override
-    public Object getObject() {
-        return object;
-    }
-
-    @Override
-    public void setObject(Object object) {
-        this.object = object;
     }
 
 }
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
index a38a96c..91c0a8b 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
@@ -18,22 +18,10 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.expressions;
 
-public class IndexedNLJoinExpressionAnnotation implements 
IExpressionAnnotation {
+public class IndexedNLJoinExpressionAnnotation extends 
AbstractExpressionAnnotation {
 
-    public static final String INDEXED_NL_JOIN_ANNOTATION_KEY = "indexnl";
+    public static final String HINT_STRING = "indexnl";
     public static final IndexedNLJoinExpressionAnnotation INSTANCE = new 
IndexedNLJoinExpressionAnnotation();
-
-    private Object object;
-
-    @Override
-    public Object getObject() {
-        return object;
-    }
-
-    @Override
-    public void setObject(Object object) {
-        this.object = object;
-    }
 
     @Override
     public IExpressionAnnotation copy() {
@@ -44,6 +32,6 @@
 
     @Override
     public String toString() {
-        return INDEXED_NL_JOIN_ANNOTATION_KEY;
+        return HINT_STRING;
     }
 }
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
index 1a7e224..4686f32 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractLogicalOperator.java
@@ -43,32 +43,34 @@
 
 public abstract class AbstractLogicalOperator implements ILogicalOperator {
 
-    /*********************************************************************
-     * UNPARTITIONED, the input data is not partitioned
-     * PARTITIONED, the input data is partitioned, the operator is executed on
-     * each partition and may receive input from other partitions (e.g. if it 
is
-     * a join or an aggregate)
-     * LOCAL, the input data is partitioned, the operator is executed on each
-     * partition and only processes data from that partition
-     */
-
-    public static enum ExecutionMode {
+    public enum ExecutionMode {
+        /**
+         * UNPARTITIONED, the input data is not partitioned
+         */
         UNPARTITIONED,
+        /**
+         * PARTITIONED, the input data is partitioned, the operator is 
executed on
+         * each partition and may receive input from other partitions (e.g. if 
it is
+         * a join or an aggregate)
+         */
         PARTITIONED,
+        /**
+         * LOCAL, the input data is partitioned, the operator is executed on 
each
+         * partition and only processes data from that partition
+         */
         LOCAL
     }
 
     private AbstractLogicalOperator.ExecutionMode mode = 
AbstractLogicalOperator.ExecutionMode.UNPARTITIONED;
     protected IPhysicalOperator physicalOperator;
-    private final Map<String, Object> annotations = new HashMap<String, 
Object>();
+    private final Map<String, Object> annotations = new HashMap<>();
     private boolean bJobGenEnabled = true;
 
-    final protected List<Mutable<ILogicalOperator>> inputs;
-    // protected List<LogicalOperatorReference> outputs;
+    protected final List<Mutable<ILogicalOperator>> inputs;
     protected List<LogicalVariable> schema;
 
     public AbstractLogicalOperator() {
-        inputs = new ArrayList<Mutable<ILogicalOperator>>();
+        inputs = new ArrayList<>();
     }
 
     @Override
@@ -134,11 +136,6 @@
         return inputs;
     }
 
-    // @Override
-    // public final List<LogicalOperatorReference> getOutputs() {
-    // return outputs;
-    // }
-
     @Override
     public final boolean hasInputs() {
         return !inputs.isEmpty();
@@ -161,7 +158,7 @@
     @Override
     public final void contributeRuntimeOperator(IHyracksJobBuilder builder, 
JobGenContext context,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, 
IOperatorSchema outerPlanSchema)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         if (bJobGenEnabled) {
             if (physicalOperator == null) {
                 throw new AlgebricksException("Physical operator not set for 
operator: " + this);
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java
index 343ace8..834107c 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ReplicateOperator.java
@@ -34,20 +34,20 @@
 
 public class ReplicateOperator extends AbstractLogicalOperator {
 
-    private int outputArity = 2;
-    private boolean[] outputMaterializationFlags = new boolean[outputArity];
+    private int outputArity;
+    private boolean[] outputMaterializationFlags;
     private List<Mutable<ILogicalOperator>> outputs;
 
     public ReplicateOperator(int outputArity) {
         this.outputArity = outputArity;
         this.outputMaterializationFlags = new boolean[outputArity];
-        this.outputs = new ArrayList<Mutable<ILogicalOperator>>();
+        this.outputs = new ArrayList<>();
     }
 
     public ReplicateOperator(int outputArity, boolean[] 
outputMaterializationFlags) {
         this.outputArity = outputArity;
         this.outputMaterializationFlags = outputMaterializationFlags;
-        this.outputs = new ArrayList<Mutable<ILogicalOperator>>();
+        this.outputs = new ArrayList<>();
     }
 
     @Override
@@ -87,10 +87,6 @@
 
     public int getOutputArity() {
         return outputArity;
-    }
-
-    public int setOutputArity(int outputArity) {
-        return this.outputArity = outputArity;
     }
 
     public void setOutputMaterializationFlags(boolean[] 
outputMaterializationFlags) {
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java
index aad0cf3..799a6af 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractExchangePOperator.java
@@ -29,6 +29,7 @@
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
 
 public abstract class AbstractExchangePOperator extends 
AbstractPhysicalOperator {
+    @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, 
JobGenContext context, ILogicalOperator op,
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, 
IOperatorSchema outerPlanSchema)
             throws AlgebricksException {
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
index d9e1540..835cc14 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
@@ -37,7 +37,7 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import 
org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import 
org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
 
 public class BroadcastExchangePOperator extends AbstractExchangePOperator {
 
@@ -68,8 +68,8 @@
     @Override
     public Pair<IConnectorDescriptor, TargetConstraint> 
createConnectorDescriptor(IConnectorDescriptorRegistry spec,
             ILogicalOperator op, IOperatorSchema opSchema, JobGenContext 
context) throws AlgebricksException {
-        IConnectorDescriptor conn = new 
MToNReplicatingConnectorDescriptor(spec);
-        return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+        IConnectorDescriptor conn = new MToNBroadcastConnectorDescriptor(spec);
+        return new Pair<>(conn, null);
     }
 
 }
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
index 818e1ec..083e4d3 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/OneToOneExchangePOperator.java
@@ -35,9 +35,6 @@
 
 public class OneToOneExchangePOperator extends AbstractExchangePOperator {
 
-    public OneToOneExchangePOperator() {
-    }
-
     @Override
     public PhysicalOperatorTag getOperatorTag() {
         return PhysicalOperatorTag.ONE_TO_ONE_EXCHANGE;
@@ -59,7 +56,7 @@
     public Pair<IConnectorDescriptor, TargetConstraint> 
createConnectorDescriptor(IConnectorDescriptorRegistry spec,
             ILogicalOperator op, IOperatorSchema opSchema, JobGenContext 
context) {
         IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
-        return new Pair<IConnectorDescriptor, TargetConstraint>(conn, 
TargetConstraint.SAME_COUNT);
+        return new Pair<>(conn, TargetConstraint.SAME_COUNT);
     }
 
 }
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
index e11a64f..aedf046 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
@@ -34,7 +34,7 @@
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import 
org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import 
org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
 
 public class RandomMergeExchangePOperator extends AbstractExchangePOperator {
 
@@ -58,7 +58,7 @@
     @Override
     public Pair<IConnectorDescriptor, TargetConstraint> 
createConnectorDescriptor(IConnectorDescriptorRegistry spec,
             ILogicalOperator op, IOperatorSchema opSchema, JobGenContext 
context) {
-        IConnectorDescriptor conn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor conn = new MToNBroadcastConnectorDescriptor(spec);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, 
TargetConstraint.ONE);
     }
 }
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
index 14a8f16..db778f7 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
@@ -89,7 +89,7 @@
                 outputDependencyLabels[i] = 1;
             }
         }
-        return new Pair<int[], int[]>(inputDependencyLabels, 
outputDependencyLabels);
+        return new Pair<>(inputDependencyLabels, outputDependencyLabels);
     }
 
     @Override
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
index e6517d0..823294e 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
@@ -21,15 +21,12 @@
 import java.util.ArrayList;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import 
org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import 
org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import 
org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import 
org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import 
org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
 import 
org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
index 0b7cb3e..20e6215 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
@@ -26,11 +26,30 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 
 public interface IPartitioningProperty extends IStructuralProperty {
-    enum PartitioningType {
+    /**
+     * The Partitioning Types define the method data is transfered between 
partitions and/or properties of the data.
+     */
+    public enum PartitioningType {
+        /**
+         * Data is not partitioned.
+         */
         UNPARTITIONED,
+        /**
+         * Data is partitioned without a repeatable method.
+         */
         RANDOM,
+        /**
+         * Data is replicated to all partitions.
+         */
         BROADCAST,
+        /**
+         * Data is hash partitioned.
+         */
         UNORDERED_PARTITIONED,
+        /**
+         * Data is range partitioned (only used on data that has a total 
order).
+         * The partitions are order based on the data range.
+         */
         ORDERED_PARTITIONED
     }
 
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
index fee9174..9d2a5da 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/typing/PropagatingTypeEnvironment.java
@@ -36,9 +36,9 @@
 
     private final ITypeEnvPointer[] envPointers;
 
-    private final List<LogicalVariable> nonNullVariables = new 
ArrayList<LogicalVariable>();
+    private final List<LogicalVariable> nonNullVariables = new ArrayList<>();
 
-    private final List<List<LogicalVariable>> correlatedNullableVariableLists 
= new ArrayList<List<LogicalVariable>>();
+    private final List<List<LogicalVariable>> correlatedNullableVariableLists 
= new ArrayList<>();
 
     public PropagatingTypeEnvironment(IExpressionTypeComputer 
expressionTypeComputer,
             IMissableTypeComputer nullableTypeComputer, IMetadataProvider<?, 
?> metadataProvider,
diff --git 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
index af40250..a1c6164 100644
--- 
a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
+++ 
b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobBuilder.java
@@ -49,19 +49,19 @@
     private final AlgebricksAbsolutePartitionConstraint clusterLocations;
     private final AlgebricksAbsolutePartitionConstraint countOneLocation;
 
-    private final Map<ILogicalOperator, ArrayList<ILogicalOperator>> outEdges 
= new HashMap<ILogicalOperator, ArrayList<ILogicalOperator>>();
-    private final Map<ILogicalOperator, ArrayList<ILogicalOperator>> inEdges = 
new HashMap<ILogicalOperator, ArrayList<ILogicalOperator>>();
-    private final Map<ILogicalOperator, Pair<IConnectorDescriptor, 
TargetConstraint>> connectors = new HashMap<ILogicalOperator, 
Pair<IConnectorDescriptor, TargetConstraint>>();
+    private final Map<ILogicalOperator, ArrayList<ILogicalOperator>> outEdges 
= new HashMap<>();
+    private final Map<ILogicalOperator, ArrayList<ILogicalOperator>> inEdges = 
new HashMap<>();
+    private final Map<ILogicalOperator, Pair<IConnectorDescriptor, 
TargetConstraint>> connectors = new HashMap<>();
 
-    private final Map<ILogicalOperator, Pair<IPushRuntimeFactory, 
RecordDescriptor>> microOps = new HashMap<ILogicalOperator, 
Pair<IPushRuntimeFactory, RecordDescriptor>>();
-    private final Map<IPushRuntimeFactory, ILogicalOperator> revMicroOpMap = 
new HashMap<IPushRuntimeFactory, ILogicalOperator>();
-    private final Map<ILogicalOperator, IOperatorDescriptor> hyracksOps = new 
HashMap<ILogicalOperator, IOperatorDescriptor>();
-    private final Map<ILogicalOperator, AlgebricksPartitionConstraint> 
pcForMicroOps = new HashMap<ILogicalOperator, AlgebricksPartitionConstraint>();
+    private final Map<ILogicalOperator, Pair<IPushRuntimeFactory, 
RecordDescriptor>> microOps = new HashMap<>();
+    private final Map<IPushRuntimeFactory, ILogicalOperator> revMicroOpMap = 
new HashMap<>();
+    private final Map<ILogicalOperator, IOperatorDescriptor> hyracksOps = new 
HashMap<>();
+    private final Map<ILogicalOperator, AlgebricksPartitionConstraint> 
pcForMicroOps = new HashMap<>();
 
-    private final Map<ILogicalOperator, Integer> 
algebraicOpBelongingToMetaAsterixOp = new HashMap<ILogicalOperator, Integer>();
-    private final Map<Integer, List<Pair<IPushRuntimeFactory, 
RecordDescriptor>>> metaAsterixOpSkeletons = new HashMap<Integer, 
List<Pair<IPushRuntimeFactory, RecordDescriptor>>>();
-    private final Map<Integer, AlgebricksMetaOperatorDescriptor> 
metaAsterixOps = new HashMap<Integer, AlgebricksMetaOperatorDescriptor>();
-    private final Map<IOperatorDescriptor, AlgebricksPartitionConstraint> 
partitionConstraintMap = new HashMap<IOperatorDescriptor, 
AlgebricksPartitionConstraint>();
+    private final Map<ILogicalOperator, Integer> 
algebraicOpBelongingToMetaAsterixOp = new HashMap<>();
+    private final Map<Integer, List<Pair<IPushRuntimeFactory, 
RecordDescriptor>>> metaAsterixOpSkeletons = new HashMap<>();
+    private final Map<Integer, AlgebricksMetaOperatorDescriptor> 
metaAsterixOps = new HashMap<>();
+    private final Map<IOperatorDescriptor, AlgebricksPartitionConstraint> 
partitionConstraintMap = new HashMap<>();
 
     private int aodCounter = 0;
 
@@ -123,14 +123,14 @@
             int destInputIndex) {
         ArrayList<ILogicalOperator> outputs = outEdges.get(src);
         if (outputs == null) {
-            outputs = new ArrayList<ILogicalOperator>();
+            outputs = new ArrayList<>();
             outEdges.put(src, outputs);
         }
         addAtPos(outputs, dest, srcOutputIndex);
 
         ArrayList<ILogicalOperator> inp = inEdges.get(dest);
         if (inp == null) {
-            inp = new ArrayList<ILogicalOperator>();
+            inp = new ArrayList<>();
             inEdges.put(dest, inp);
         }
         addAtPos(inp, src, destInputIndex);
@@ -270,7 +270,7 @@
     }
 
     private Map<IConnectorDescriptor, TargetConstraint> setupConnectors() 
throws AlgebricksException {
-        Map<IConnectorDescriptor, TargetConstraint> tgtConstraints = new 
HashMap<IConnectorDescriptor, TargetConstraint>();
+        Map<IConnectorDescriptor, TargetConstraint> tgtConstraints = new 
HashMap<>();
         for (ILogicalOperator exchg : connectors.keySet()) {
             ILogicalOperator inOp = inEdges.get(exchg).get(0);
             ILogicalOperator outOp = outEdges.get(exchg).get(0);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
index 7082e77..d1fc8a2 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
@@ -27,7 +27,7 @@
 public class PointableBinaryComparatorFactory implements 
IBinaryComparatorFactory {
     private static final long serialVersionUID = 1L;
 
-    private final IPointableFactory pf;
+    protected final IPointableFactory pf;
 
     public static PointableBinaryComparatorFactory of(IPointableFactory pf) {
         return new PointableBinaryComparatorFactory(pf);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
index a0c4a4a..b328f29 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
@@ -159,6 +159,10 @@
             }
         }
 
+        @Override
+        public void flush() throws HyracksDataException {
+        }
+
     }
 
     private class ReaderMapperOperator extends MapperBaseOperator {
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index 2467c2a..7764265 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -215,7 +215,7 @@
                                     .appendToWriter(writer, appender, 
tb.getFieldEndOffsets(), tb.getByteArray(),
                                             0, tb.getSize());
                         }
-                        appender.flush(writer, true);
+                        appender.write(writer, true);
                     } catch (Exception e) {
                         writer.fail();
                         throw new HyracksDataException(e);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
index 1ebebda..85cd34d 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
@@ -157,7 +157,7 @@
                             }
                             tb.addFieldEndOffset();
                             if (!appender.append(tb.getFieldEndOffsets(), 
tb.getByteArray(), 0, tb.getSize())) {
-                                appender.flush(writer, true);
+                                appender.write(writer, true);
                                 if (!appender.append(tb.getFieldEndOffsets(), 
tb.getByteArray(), 0, tb.getSize())) {
                                     throw new IllegalStateException();
                                 }
@@ -167,7 +167,7 @@
 
                     @Override
                     public void close() throws HyracksDataException {
-                        appender.flush(writer, true);
+                        appender.write(writer, true);
                     }
 
                     @Override
@@ -198,7 +198,7 @@
                             value.write(dos);
                             tb.addFieldEndOffset();
                             if (!fta.append(tb.getFieldEndOffsets(), 
tb.getByteArray(), 0, tb.getSize())) {
-                                fta.flush(outputWriter, true);
+                                fta.write(outputWriter, true);
                                 if (!fta.append(tb.getFieldEndOffsets(), 
tb.getByteArray(), 0, tb.getSize())) {
                                     throw new IllegalStateException();
                                 }
@@ -207,7 +207,7 @@
 
                         @Override
                         public void close(TaskAttemptContext context) throws 
IOException, InterruptedException {
-                            fta.flush(outputWriter, true);
+                            fta.write(outputWriter, true);
                         }
                     };
                     delegatingWriter = new ReduceWriter<K2, V2, K2, V2>(ctx, 
helper,
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractStateObject.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractStateObject.java
index c4e8ec6..382ca57 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractStateObject.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractStateObject.java
@@ -18,6 +18,10 @@
  */
 package org.apache.hyracks.dataflow.std.base;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
 import org.apache.hyracks.api.dataflow.state.IStateObject;
 import org.apache.hyracks.api.job.JobId;
 
@@ -62,4 +66,12 @@
     public void setMemoryOccupancy(long memoryOccupancy) {
         this.memoryOccupancy = memoryOccupancy;
     }
+
+    @Override
+    public void toBytes(DataOutput out) throws IOException {
+    }
+
+    @Override
+    public void fromBytes(DataInput in) throws IOException {
+    }
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java
index 3683354..2717d7a 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTuplePointerAccessor.java
@@ -35,7 +35,7 @@
     @Override
     public void reset(TuplePointer tuplePointer) {
         resetInnerAccessor(tuplePointer);
-        tid = tuplePointer.tupleIndex;
+        tid = tuplePointer.getTupleIndex();
     }
 
     @Override
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
index 70ca2dd..a0a9ab0 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
@@ -86,7 +86,7 @@
      */
     void close();
 
-    ITuplePointerAccessor getTupleAccessor(RecordDescriptor recordDescriptor);
+    ITuplePointerAccessor getTuplePointerAccessor(RecordDescriptor 
recordDescriptor);
 
     /**
      * Flush the particular partition {@code pid} to {@code writer}.
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
index 324401a..94fc643 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
@@ -43,5 +43,5 @@
 
     void close() throws HyracksDataException;
 
-    ITuplePointerAccessor createTupleAccessor();
+    ITuplePointerAccessor createTuplePointerAccessor();
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
index f46a374..c193c3b 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
@@ -242,7 +242,7 @@
     }
 
     @Override
-    public ITuplePointerAccessor getTupleAccessor(final RecordDescriptor 
recordDescriptor) {
+    public ITuplePointerAccessor getTuplePointerAccessor(final 
RecordDescriptor recordDescriptor) {
         return new AbstractTuplePointerAccessor() {
             FrameTupleAccessor innerAccessor = new 
FrameTupleAccessor(recordDescriptor);
 
@@ -253,8 +253,8 @@
 
             @Override
             void resetInnerAccessor(TuplePointer tuplePointer) {
-                partitionArray[parsePartitionId(tuplePointer.frameIndex)]
-                        
.getFrame(parseFrameIdInPartition(tuplePointer.frameIndex), tempInfo);
+                partitionArray[parsePartitionId(tuplePointer.getFrameIndex())]
+                        
.getFrame(parseFrameIdInPartition(tuplePointer.getFrameIndex()), tempInfo);
                 innerAccessor.reset(tempInfo.getBuffer(), 
tempInfo.getStartOffset(), tempInfo.getLength());
             }
         };
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 4359e49..e229a3c 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
@@ -150,8 +150,8 @@
 
     @Override
     public void deleteTuple(TuplePointer tuplePointer) throws 
HyracksDataException {
-        accessor.reset(frames.get(tuplePointer.frameIndex));
-        accessor.delete(tuplePointer.tupleIndex);
+        accessor.reset(frames.get(tuplePointer.getFrameIndex()));
+        accessor.delete(tuplePointer.getTupleIndex());
         numTuples--;
     }
 
@@ -168,7 +168,7 @@
     }
 
     @Override
-    public ITuplePointerAccessor createTupleAccessor() {
+    public ITuplePointerAccessor createTuplePointerAccessor() {
         return new AbstractTuplePointerAccessor() {
             private IAppendDeletableFrameTupleAccessor bufferAccessor = new 
DeletableFrameTupleAppender(
                     recordDescriptor);
@@ -180,7 +180,7 @@
 
             @Override
             void resetInnerAccessor(TuplePointer tuplePointer) {
-                bufferAccessor.reset(frames.get(tuplePointer.frameIndex));
+                bufferAccessor.reset(frames.get(tuplePointer.getFrameIndex()));
             }
         };
     }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
similarity index 96%
rename from 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
rename to 
hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
index 7d60ce5..2ca70da 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
@@ -33,8 +33,8 @@
 import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
 import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
 
-public class MToNReplicatingConnectorDescriptor extends 
AbstractMToNConnectorDescriptor {
-    public MToNReplicatingConnectorDescriptor(IConnectorDescriptorRegistry 
spec) {
+public class MToNBroadcastConnectorDescriptor extends 
AbstractMToNConnectorDescriptor {
+    public MToNBroadcastConnectorDescriptor(IConnectorDescriptorRegistry spec) 
{
         super(spec);
     }
 
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index e08041d..f08d27d 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -106,7 +106,7 @@
                     
PreferToSpillFullyOccupiedFramePolicy.createAtMostOneFrameForSpilledPartitionConstrain(spilledSet),
                     numPartitions, framesLimit * ctx.getInitialFrameSize());
 
-            final ITuplePointerAccessor bufferAccessor = 
bufferManager.getTupleAccessor(outRecordDescriptor);
+            final ITuplePointerAccessor bufferAccessor = 
bufferManager.getTuplePointerAccessor(outRecordDescriptor);
 
             private final PreferToSpillFullyOccupiedFramePolicy spillPolicy = 
new PreferToSpillFullyOccupiedFramePolicy(
                     bufferManager, spilledSet, ctx.getInitialFrameSize());
@@ -147,7 +147,7 @@
                     bufferAccessor.reset(pointer);
                     int c = ftpcInputCompareToAggregate.compare(accessor, 
tIndex, bufferAccessor);
                     if (c == 0) {
-                        aggregateExistingTuple(accessor, tIndex, 
bufferAccessor, pointer.tupleIndex);
+                        aggregateExistingTuple(accessor, tIndex, 
bufferAccessor, pointer.getTupleIndex());
                         return true;
                     }
                 }
@@ -201,11 +201,11 @@
                         switch (type) {
                             case PARTIAL:
                                 hasOutput = 
aggregator.outputPartialResult(outputTupleBuilder, bufferAccessor,
-                                        pointer.tupleIndex, aggregateState);
+                                        pointer.getTupleIndex(), 
aggregateState);
                                 break;
                             case FINAL:
                                 hasOutput = 
aggregator.outputFinalResult(outputTupleBuilder, bufferAccessor,
-                                        pointer.tupleIndex, aggregateState);
+                                        pointer.getTupleIndex(), 
aggregateState);
                                 break;
                         }
 
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
index 486821b..c2b69a6 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -109,8 +109,7 @@
         int tCount = accessorBuild.getTupleCount();
         for (int i = 0; i < tCount; ++i) {
             int entry = tpcBuild.partition(accessorBuild, i, tableSize);
-            storedTuplePointer.frameIndex = bIndex;
-            storedTuplePointer.tupleIndex = i;
+            storedTuplePointer.reset(bIndex, i);
             table.insert(entry, storedTuplePointer);
         }
     }
@@ -123,10 +122,10 @@
             int offset = 0;
             do {
                 table.getTuplePointer(entry, offset++, storedTuplePointer);
-                if (storedTuplePointer.frameIndex < 0)
+                if (storedTuplePointer.getFrameIndex() < 0)
                     break;
-                int bIndex = storedTuplePointer.frameIndex;
-                int tIndex = storedTuplePointer.tupleIndex;
+                int bIndex = storedTuplePointer.getFrameIndex();
+                int tIndex = storedTuplePointer.getTupleIndex();
                 accessorBuild.reset(buffers.get(bIndex));
                 int c = tpComparator.compare(accessorProbe, tid, 
accessorBuild, tIndex);
                 if (c == 0) {
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
index 9ce59ae..56af78e 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -139,13 +139,6 @@
                 }
 
                 @Override
-                public void flush() throws HyracksDataException {
-                    for (int i = 0; i < numberOfNonMaterializedOutputs; i++) {
-                        writers[i].flush();
-                    }
-                }
-
-                @Override
                 public void close() throws HyracksDataException {
                     HyracksDataException hde = null;
                     try {
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
index f1f1b1b..ed28560 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
@@ -110,10 +110,10 @@
     }
 
     private void copy(int src, int dest) {
-        tPointersTemp[dest * 4] = tPointers[src * 4];
-        tPointersTemp[dest * 4 + 1] = tPointers[src * 4 + 1];
-        tPointersTemp[dest * 4 + 2] = tPointers[src * 4 + 2];
-        tPointersTemp[dest * 4 + 3] = tPointers[src * 4 + 3];
+        tPointersTemp[dest * 4 + ID_FRAMEID] = tPointers[src * 4 + ID_FRAMEID];
+        tPointersTemp[dest * 4 + ID_TUPLE_START] = tPointers[src * 4 + 
ID_TUPLE_START];
+        tPointersTemp[dest * 4 + ID_TUPLE_END] = tPointers[src * 4 + 
ID_TUPLE_END];
+        tPointersTemp[dest * 4 + ID_NORMAL_KEY] = tPointers[src * 4 + 
ID_NORMAL_KEY];
     }
 
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
index a5ec88a..c473819 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
@@ -125,8 +125,8 @@
             INormalizedKeyComputerFactory firstKeyNormalizerFactory, 
IBinaryComparatorFactory[] comparatorFactories)
             throws HyracksDataException {
         this.bufferManager = bufferManager;
-        this.bufferAccessor1 = bufferManager.createTupleAccessor();
-        this.bufferAccessor2 = bufferManager.createTupleAccessor();
+        this.bufferAccessor1 = bufferManager.createTuplePointerAccessor();
+        this.bufferAccessor2 = bufferManager.createTuplePointerAccessor();
         this.topK = topK;
         this.outputFrame = new VSizeFrame(ctx);
         this.outputAppender = new FrameTupleAppender();
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
index 8699c10..b42cdb7 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -34,8 +34,8 @@
     private static final int INIT_ENTRY_SIZE = 4;
 
     private IntSerDeBuffer[] headers;
-    private List<IntSerDeBuffer> contents = new ArrayList<IntSerDeBuffer>();
-    private List<Integer> frameCurrentIndex = new ArrayList<Integer>();
+    private List<IntSerDeBuffer> contents = new ArrayList<>();
+    private List<Integer> frameCurrentIndex = new ArrayList<>();
     private final IHyracksFrameMgrContext ctx;
     private final int frameCapacity;
     private int currentLargestFrameIndex = 0;
@@ -103,22 +103,19 @@
         int headerOffset = getHeaderFrameOffset(entry);
         IntSerDeBuffer header = headers[hFrameIndex];
         if (header == null) {
-            dataPointer.frameIndex = -1;
-            dataPointer.tupleIndex = -1;
+            dataPointer.reset(-1, -1);
             return false;
         }
         int frameIndex = header.getInt(headerOffset);
         int offsetIndex = header.getInt(headerOffset + 1);
         if (frameIndex < 0) {
-            dataPointer.frameIndex = -1;
-            dataPointer.tupleIndex = -1;
+            dataPointer.reset(-1, -1);
             return false;
         }
         IntSerDeBuffer frame = contents.get(frameIndex);
         int entryUsedItems = frame.getInt(offsetIndex + 1);
         if (offset > entryUsedItems - 1) {
-            dataPointer.frameIndex = -1;
-            dataPointer.tupleIndex = -1;
+            dataPointer.reset(-1, -1);
             return false;
         }
         int startIndex = offsetIndex + 2 + offset * 2;
@@ -127,8 +124,7 @@
             startIndex -= frameCapacity;
         }
         frame = contents.get(frameIndex);
-        dataPointer.frameIndex = frame.getInt(startIndex);
-        dataPointer.tupleIndex = frame.getInt(startIndex + 1);
+        dataPointer.reset(frame.getInt(startIndex), frame.getInt(startIndex + 
1));
         return true;
     }
 
@@ -152,6 +148,7 @@
         return headerFrameCount + contents.size();
     }
 
+    @Override
     public int getTupleCount() {
         return tupleCount;
     }
@@ -218,8 +215,8 @@
         // set the entry
         lastFrame.writeInt(lastIndex, entryCapacity - 1);
         lastFrame.writeInt(lastIndex + 1, 1);
-        lastFrame.writeInt(lastIndex + 2, pointer.frameIndex);
-        lastFrame.writeInt(lastIndex + 3, pointer.tupleIndex);
+        lastFrame.writeInt(lastIndex + 2, pointer.getFrameIndex());
+        lastFrame.writeInt(lastIndex + 3, pointer.getTupleIndex());
         int newLastIndex = lastIndex + entryCapacity * 2;
         newLastIndex = newLastIndex < frameCapacity ? newLastIndex : 
frameCapacity - 1;
         frameCurrentIndex.set(startFrameIndex, newLastIndex);
@@ -233,8 +230,9 @@
         }
     }
 
-    private void insertNonFirstTuple(IntSerDeBuffer header, int headerOffset, 
int frameIndex, int offsetIndex,
+    private void insertNonFirstTuple(IntSerDeBuffer header, int headerOffset, 
int frameIndexArg, int offsetIndex,
             TuplePointer pointer) throws HyracksDataException {
+        int frameIndex = frameIndexArg;
         IntSerDeBuffer frame = contents.get(frameIndex);
         int entryItems = frame.getInt(offsetIndex);
         int entryUsedItems = frame.getInt(offsetIndex + 1);
@@ -247,16 +245,15 @@
                 startIndex -= frameCapacity;
             }
             frame = contents.get(frameIndex);
-            frame.writeInt(startIndex, pointer.frameIndex);
-            frame.writeInt(startIndex + 1, pointer.tupleIndex);
+            frame.writeInt(startIndex, pointer.getFrameIndex());
+            frame.writeInt(startIndex + 1, pointer.getTupleIndex());
         } else {
             int capacity = (entryItems + 1) * 2;
             header.writeInt(headerOffset, -1);
             header.writeInt(headerOffset + 1, -1);
             int fIndex = frame.getInt(offsetIndex + 2);
             int tIndex = frame.getInt(offsetIndex + 3);
-            tempTuplePointer.frameIndex = fIndex;
-            tempTuplePointer.tupleIndex = tIndex;
+            tempTuplePointer.reset(fIndex, tIndex);
             this.insertNewEntry(header, headerOffset, capacity, 
tempTuplePointer);
             int newFrameIndex = header.getInt(headerOffset);
             int newTupleIndex = header.getInt(headerOffset + 1);
@@ -271,8 +268,7 @@
                 frame = contents.get(startFrameIndex);
                 fIndex = frame.getInt(startIndex);
                 tIndex = frame.getInt(startIndex + 1);
-                tempTuplePointer.frameIndex = fIndex;
-                tempTuplePointer.tupleIndex = tIndex;
+                tempTuplePointer.reset(fIndex, tIndex);
                 insertNonFirstTuple(header, headerOffset, newFrameIndex, 
newTupleIndex, tempTuplePointer);
             }
             insertNonFirstTuple(header, headerOffset, newFrameIndex, 
newTupleIndex, pointer);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
index 71f9639..ea46bff 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
@@ -20,8 +20,8 @@
 
 public class TuplePointer implements IResetable<TuplePointer> {
     public static final int INVALID_ID = -1;
-    public int frameIndex;
-    public int tupleIndex;
+    private int frameIndex;
+    private int tupleIndex;
 
     public TuplePointer() {
         this(INVALID_ID, INVALID_ID);
@@ -31,6 +31,15 @@
         reset(frameId, tupleId);
     }
 
+    public int getFrameIndex() {
+        return frameIndex;
+    }
+
+    public int getTupleIndex() {
+        return tupleIndex;
+    }
+
+    @Override
     public void reset(TuplePointer other) {
         reset(other.frameIndex, other.tupleIndex);
     }
@@ -62,4 +71,9 @@
         this.tupleIndex = tupleId;
     }
 
+    @Override
+    public String toString() {
+        return "TuplePointer(" + frameIndex + ", " + tupleIndex + ")";
+    }
+
 }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
index a705222..c58a2fa 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
@@ -47,10 +47,10 @@
     FrameTupleAccessor inFTA = new FrameTupleAccessor(recordDescriptor);
     Random random = new Random(System.currentTimeMillis());
 
-    abstract ITuplePointerAccessor getTupleAccessor();
+    abstract ITuplePointerAccessor getTuplePointerAccessor();
 
     protected void assertEachTupleInFTAIsInBuffer(Map<Integer, Integer> map, 
Map<TuplePointer, Integer> mapInserted) {
-        ITuplePointerAccessor accessor = getTupleAccessor();
+        ITuplePointerAccessor accessor = getTuplePointerAccessor();
         for (Map.Entry<TuplePointer, Integer> entry : mapInserted.entrySet()) {
             accessor.reset(entry.getKey());
             int dataLength = map.get(entry.getValue());
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
index d73b9ae..726b654 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
@@ -153,7 +153,7 @@
     }
 
     @Override
-    ITuplePointerAccessor getTupleAccessor() {
-        return bufferManager.getTupleAccessor(recordDescriptor);
+    ITuplePointerAccessor getTuplePointerAccessor() {
+        return bufferManager.getTuplePointerAccessor(recordDescriptor);
     }
 }
\ No newline at end of file
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
index 618dc6d..e2a231f 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
@@ -108,8 +108,8 @@
     }
 
     @Override
-    ITuplePointerAccessor getTupleAccessor() {
-        return tupleMemoryManager.createTupleAccessor();
+    ITuplePointerAccessor getTuplePointerAccessor() {
+        return tupleMemoryManager.createTuplePointerAccessor();
     }
 
     private Map<TuplePointer, Integer> insertInFTAToBufferShouldAllSuccess() 
throws HyracksDataException {
@@ -165,7 +165,7 @@
         assert (minNumOfRecordTobeDeleted < mapInserted.size());
         int countDeleted = minNumOfRecordTobeDeleted + 
random.nextInt(mapInserted.size() - minNumOfRecordTobeDeleted);
 
-        ITuplePointerAccessor accessor = 
tupleMemoryManager.createTupleAccessor();
+        ITuplePointerAccessor accessor = 
tupleMemoryManager.createTuplePointerAccessor();
         for (int i = 0; i < countDeleted; i++) {
             Iterator<Map.Entry<TuplePointer, Integer>> iter = 
mapInserted.entrySet().iterator();
             assert (iter.hasNext());
diff --git 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java
 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java
index 5893f35..43a1f6d 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/structures/SerializableHashTableTest.java
@@ -70,7 +70,7 @@
         int loop = 0;
         for (int i = 0; i < num; i++) {
             assertTrue(nsTable.getTuplePointer(i % NUM_PART, loop, pointer));
-            assertTrue(pointer.frameIndex == i);
+            assertTrue(pointer.getFrameIndex() == i);
             if (i % NUM_PART == NUM_PART - 1) {
                 loop++;
             }
diff --git 
a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
 
b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
index da674a4..d12d534 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
@@ -42,7 +42,7 @@
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import 
org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import 
org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import 
org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import 
org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -124,7 +124,7 @@
         IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn4, sorter2, 0, group2, 0);
 
-        IConnectorDescriptor conn5 = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor conn5 = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(conn5, group2, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -197,7 +197,7 @@
         IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn4, sorter2, 0, group2, 0);
 
-        IConnectorDescriptor conn5 = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor conn5 = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(conn5, group2, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -270,7 +270,7 @@
         IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
         spec.connect(conn4, sorter2, 0, group2, 0);
 
-        IConnectorDescriptor conn5 = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor conn5 = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(conn5, group2, 0, printer, 0);
 
         spec.addRoot(printer);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
 
b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index c8e6f59..43ef74d 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -41,7 +41,7 @@
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
 import 
org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
 import 
org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import 
org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import 
org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -655,7 +655,7 @@
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -746,7 +746,7 @@
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -837,7 +837,7 @@
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -924,7 +924,7 @@
                                 .of(UTF8StringPointable.FACTORY) }));
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -1023,7 +1023,7 @@
         IConnectorDescriptor custJoinConn = new 
OneToOneConnectorDescriptor(spec);
         spec.connect(custJoinConn, custMat, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
 
b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index c568306..3cd34f3 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -43,7 +43,7 @@
 import 
org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
 import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import 
org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import 
org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -189,7 +189,7 @@
         IConnectorDescriptor ordJoinConn = new 
OneToOneConnectorDescriptor(spec);
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor custJoinConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
         IConnectorDescriptor joinPrinterConn = new 
OneToOneConnectorDescriptor(spec);
@@ -266,10 +266,10 @@
         IConnectorDescriptor ordJoinConn = new 
OneToOneConnectorDescriptor(spec);
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor custJoinConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -343,10 +343,10 @@
         IConnectorDescriptor ordJoinConn = new 
OneToOneConnectorDescriptor(spec);
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor custJoinConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
@@ -425,10 +425,10 @@
         IConnectorDescriptor ordJoinConn = new 
OneToOneConnectorDescriptor(spec);
         spec.connect(ordJoinConn, ordScanner, 0, join, 0);
 
-        IConnectorDescriptor custJoinConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor custJoinConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(custJoinConn, custScanner, 0, join, 1);
 
-        IConnectorDescriptor joinPrinterConn = new 
MToNReplicatingConnectorDescriptor(spec);
+        IConnectorDescriptor joinPrinterConn = new 
MToNBroadcastConnectorDescriptor(spec);
         spec.connect(joinPrinterConn, join, 0, printer, 0);
 
         spec.addRoot(printer);
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
index d18db01..b4239d7 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
@@ -35,7 +35,7 @@
     private final IBufferCache bufferCache;
     private final IMetaDataPageManager freePageManager;
     private final int fileId;
-    private final ArrayList<IntArrayList> pagesByLevel = new 
ArrayList<IntArrayList>();
+    private final ArrayList<IntArrayList> pagesByLevel = new ArrayList<>();
     private final Random rnd = new Random();
 
     public TreeIndexBufferCacheWarmup(IBufferCache bufferCache,
@@ -62,7 +62,6 @@
                     pagesByLevel.add(new IntArrayList(100, 100));
                 }
                 if (level >= 0) {
-                    // System.out.println("ADDING: " + level + " " + pageId);
                     pagesByLevel.get(level).add(pageId);
                 }
             } finally {
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
index 5a3a88f..3194674 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
@@ -20,7 +20,6 @@
 
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
-import 
org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import 
org.apache.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.common.arraylist.DoubleArrayList;
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
index 9678289..30eb991 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
@@ -20,7 +20,6 @@
 
 import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
 import org.apache.hyracks.data.std.primitive.DoublePointable;
-import 
org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
 import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
 import 
org.apache.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;
 import org.apache.hyracks.storage.common.arraylist.DoubleArrayList;
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
index fd2bdec..ec70671 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
@@ -60,8 +60,9 @@
     }
 
     public void removeLast() {
-        if (size > 0)
+        if (size > 0) {
             size--;
+        }
     }
 
     // WARNING: caller is responsible for checking size > 0
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
index 84885ca..e164b6b 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
@@ -60,8 +60,9 @@
     }
 
     public void removeLast() {
-        if (size > 0)
+        if (size > 0) {
             size--;
+        }
     }
 
     // WARNING: caller is responsible for checking size > 0
diff --git 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
index 2811641..7393abd 100644
--- 
a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
+++ 
b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
@@ -49,7 +49,6 @@
         size++;
     }
 
-
     public void add(long i) {
         if (size == data.length) {
             long[] newData = new long[data.length + growth];
@@ -61,8 +60,9 @@
     }
 
     public void removeLast() {
-        if (size > 0)
+        if (size > 0) {
             size--;
+        }
     }
 
     // WARNING: caller is responsible for checking size > 0

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ieb72a0fef3c181282cc4c71b881a7cb52da86818
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <prest...@apache.org>

Reply via email to