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

[2/2] incubator-asterixdb git commit: Refactored the AbstractComparisonEvaluator.

Refactored the AbstractComparisonEvaluator.

The refactoring allows the comparison code to be used from other locations,
specifically, for generic interval logic.

Change-Id: I42e0e8cf71207bb862334cd0629e8c024ff0556c
Reviewed-on: https://asterix-gerrit.ics.uci.edu/801
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>


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

Branch: refs/heads/master
Commit: f29b26fd0da390240486e3bbf275bdb59ba55927
Parents: 22302e7
Author: Preston Carman <pr...@apache.org>
Authored: Mon Apr 18 16:31:44 2016 -0700
Committer: Preston Carman <pr...@apache.org>
Committed: Mon Apr 18 17:13:29 2016 -0700

----------------------------------------------------------------------
 .../comparators/ABinaryComparator.java          |  12 +-
 .../nonvisitor/AIntervalPointable.java          | 162 ++++++
 .../AbstractComparisonEvaluator.java            | 553 +++----------------
 .../comparisons/ComparisonEvalFactory.java      | 292 ++--------
 .../comparisons/ComparisonHelper.java           | 486 ++++++++++++++++
 .../AbstractIntervalLogicFuncDescriptor.java    |  49 +-
 .../GetOverlappingIntervalDescriptor.java       |  62 +--
 .../temporal/IntervalAfterDescriptor.java       |  16 +-
 .../temporal/IntervalBeforeDescriptor.java      |  13 +-
 .../temporal/IntervalCoveredByDescriptor.java   |  16 +-
 .../temporal/IntervalCoversDescriptor.java      |  16 +-
 .../temporal/IntervalEndedByDescriptor.java     |  16 +-
 .../temporal/IntervalEndsDecriptor.java         |  16 +-
 .../functions/temporal/IntervalLogic.java       | 206 ++++---
 .../temporal/IntervalMeetsDescriptor.java       |  16 +-
 .../temporal/IntervalMetByDescriptor.java       |  16 +-
 .../IntervalOverlappedByDescriptor.java         |  16 +-
 .../temporal/IntervalOverlapsDescriptor.java    |  16 +-
 .../temporal/IntervalStartedByDescriptor.java   |  16 +-
 .../temporal/IntervalStartsDescriptor.java      |  16 +-
 .../functions/temporal/OverlapDescriptor.java   |  16 +-
 .../data/std/primitive/ByteArrayPointable.java  |   2 +-
 .../std/primitive/TaggedValuePointable.java     |  48 ++
 23 files changed, 1061 insertions(+), 1016 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f29b26fd/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
index ec337eb..05ffb3a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
@@ -30,19 +30,19 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
  */
 public abstract class ABinaryComparator implements IBinaryComparator {
 
-    public static ComparableResultCode isComparable(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+    public static ComparableResultCode isComparable(byte tag1,byte tag2) {
         // NULL Check. If one type is NULL, then we return NULL
-        if (b1[s1] == ATypeTag.SERIALIZED_NULL_TYPE_TAG || b2[s2] == ATypeTag.SERIALIZED_NULL_TYPE_TAG || b1[s1] == 0
-                || b2[s2] == 0) {
+        if (tag1 == ATypeTag.SERIALIZED_NULL_TYPE_TAG || tag2 == ATypeTag.SERIALIZED_NULL_TYPE_TAG || tag1 == 0
+                || tag2 == 0) {
             return ComparableResultCode.UNKNOWN;
         }
 
         // Checks whether two types are comparable or not
-        ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b1[s1]);
-        ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b2[s2]);
+        ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(tag1);
+        ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(tag2);
 
         // Are two types compatible, meaning that they can be compared? (e.g., compare between numeric types
-        if (ATypeHierarchy.isCompatible(tag1, tag2)) {
+        if (ATypeHierarchy.isCompatible(typeTag1, typeTag2)) {
             return ComparableResultCode.TRUE;
         } else {
             return ComparableResultCode.FALSE;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f29b26fd/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
new file mode 100644
index 0000000..e9b1578
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
@@ -0,0 +1,162 @@
+/*
+ * 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.asterix.om.pointables.nonvisitor;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.data.std.api.AbstractPointable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IPointableFactory;
+import org.apache.hyracks.data.std.primitive.BytePointable;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+
+/*
+ * This class interprets the binary data representation of an interval.
+ *
+ * Interval {
+ *   byte type;
+ *   IPointable start;
+ *   IPointable end;
+ * }
+ */
+public class AIntervalPointable extends AbstractPointable {
+
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return false;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 0;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new AIntervalPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static final IObjectFactory<IPointable, ATypeTag> ALLOCATOR = new IObjectFactory<IPointable, ATypeTag>() {
+        @Override
+        public IPointable create(ATypeTag type) {
+            return new AIntervalPointable();
+        }
+    };
+
+    private static final int TAG_SIZE = 1;
+
+    public byte getType() {
+        return BytePointable.getByte(bytes, getTypeOffset());
+    }
+
+    public ATypeTag getTypeTag() {
+        return EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(getType());
+    }
+
+    public int getTypeOffset() {
+        return start;
+    }
+
+    public int getTypeSize() {
+        return TAG_SIZE;
+    }
+
+    public void getStart(IPointable start) throws AsterixException {
+        start.set(bytes, getStartOffset(), getStartSize());
+    }
+
+    public int getStartOffset() {
+        return getTypeOffset() + getTypeSize();
+    }
+
+    public int getStartSize() throws AsterixException {
+        switch (getTypeTag()) {
+            case DATE:
+            case TIME:
+                return Integer.BYTES;
+            case DATETIME:
+                return Long.BYTES;
+            default:
+                throw new AsterixException("Unsupported interval type: " + getTypeTag() + ".");
+        }
+    }
+
+    public long getStartValue() throws AsterixException {
+        switch (getTypeTag()) {
+            case DATE:
+            case TIME:
+                return IntegerPointable.getInteger(bytes, getStartOffset());
+            case DATETIME:
+                return LongPointable.getLong(bytes, getStartOffset());
+            default:
+                throw new AsterixException("Unsupported interval type: " + getTypeTag() + ".");
+        }
+    }
+
+    public void getEnd(IPointable start) throws AsterixException {
+        start.set(bytes, getEndOffset(), getEndSize());
+    }
+
+    public int getEndOffset() throws AsterixException {
+        return getStartOffset() + getStartSize();
+    }
+
+    public int getEndSize() throws AsterixException {
+        switch (getTypeTag()) {
+            case DATE:
+            case TIME:
+                return Integer.BYTES;
+            case DATETIME:
+                return Long.BYTES;
+            default:
+                throw new AsterixException("Unsupported interval type: " + getTypeTag() + ".");
+        }
+    }
+
+    public long getEndValue() throws AsterixException {
+        switch (getTypeTag()) {
+            case DATE:
+            case TIME:
+                return IntegerPointable.getInteger(bytes, getEndOffset());
+            case DATETIME:
+                return LongPointable.getLong(bytes, getEndOffset());
+            default:
+                throw new AsterixException("Unsupported interval type: " + getTypeTag() + ".");
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f29b26fd/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
index c78cf6e..30f6562 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
@@ -21,22 +21,6 @@ package org.apache.asterix.runtime.evaluators.comparisons;
 import java.io.DataOutput;
 
 import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator;
-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;
-import org.apache.asterix.dataflow.data.nontagged.comparators.ALinePartialBinaryComparatorFactory;
-import org.apache.asterix.dataflow.data.nontagged.comparators.APoint3DPartialBinaryComparatorFactory;
-import org.apache.asterix.dataflow.data.nontagged.comparators.APointPartialBinaryComparatorFactory;
-import org.apache.asterix.dataflow.data.nontagged.comparators.APolygonPartialBinaryComparatorFactory;
-import org.apache.asterix.dataflow.data.nontagged.comparators.ARectanglePartialBinaryComparatorFactory;
-import org.apache.asterix.dataflow.data.nontagged.comparators.AUUIDPartialBinaryComparatorFactory;
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
-import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ANull;
@@ -48,14 +32,10 @@ import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.BinaryComparatorConstant.ComparableResultCode;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
-import org.apache.hyracks.data.std.primitive.FloatPointable;
-import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
@@ -71,39 +51,20 @@ public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
 
     protected ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     protected DataOutput out = resultStorage.getDataOutput();
-    protected IPointable outLeft = new VoidPointable();
-    protected IPointable outRight = new VoidPointable();
+    protected TaggedValuePointable argLeft = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    protected TaggedValuePointable argRight = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    protected IPointable outLeft = VoidPointable.FACTORY.createPointable();
+    protected IPointable outRight = VoidPointable.FACTORY.createPointable();
     protected IScalarEvaluator evalLeft;
     protected IScalarEvaluator evalRight;
+    private ComparisonHelper ch = new ComparisonHelper();
+
     @SuppressWarnings("unchecked")
     protected ISerializerDeserializer<ABoolean> serde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.ABOOLEAN);
     @SuppressWarnings("unchecked")
     protected ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.ANULL);
-    protected IBinaryComparator strBinaryComp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator circleBinaryComp = ACirclePartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator durationBinaryComp = ADurationPartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator intervalBinaryComp = AIntervalAscPartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator lineBinaryComparator = ALinePartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator pointBinaryComparator = APointPartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator point3DBinaryComparator = APoint3DPartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator polygonBinaryComparator = APolygonPartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator rectangleBinaryComparator = ARectanglePartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-    protected IBinaryComparator uuidBinaryComparator = AUUIDPartialBinaryComparatorFactory.INSTANCE
-            .createBinaryComparator();
-
-    protected final IBinaryComparator byteArrayComparator = new PointableBinaryComparatorFactory(
-            ByteArrayPointable.FACTORY).createBinaryComparator();
 
     public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
             IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext context) throws AlgebricksException {
@@ -111,17 +72,71 @@ public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
         this.evalRight = evalRightFactory.createScalarEvaluator(context);
     }
 
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+        resultStorage.reset();
+        evalInputs(tuple);
+
+        // checks whether we can apply >, >=, <, and <= to the given type since
+        // these operations cannot be defined for certain types.
+        if (isTotallyOrderable()) {
+            checkTotallyOrderable();
+        }
+
+        // Checks whether two types are comparable
+        switch (comparabilityCheck()) {
+            case UNKNOWN:
+                // result:UNKNOWN - NULL value found
+                try {
+                    nullSerde.serialize(ANull.NULL, out);
+                    result.set(resultStorage);
+                    return;
+                } catch (HyracksDataException e) {
+                    throw new AlgebricksException(e);
+                }
+            case FALSE:
+                // result:FALSE - two types cannot be compared. Thus we return FALSE since this is equality comparison
+                ABoolean b = ABoolean.FALSE;
+                try {
+                    serde.serialize(b, out);
+                } catch (HyracksDataException e) {
+                    throw new AlgebricksException(e);
+                }
+                break;
+            case TRUE:
+                // Two types can be compared
+                ComparisonResult r = compareResults();
+                ABoolean b1 = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
+                try {
+                    serde.serialize(b1, out);
+                } catch (HyracksDataException e) {
+                    throw new AlgebricksException(e);
+                }
+                break;
+            default:
+                throw new AlgebricksException(
+                        "Comparison cannot be processed. The return code from ComparabilityCheck is not correct.");
+        }
+        result.set(resultStorage);
+    }
+
+    protected abstract boolean isTotallyOrderable();
+
+    protected abstract boolean getComparisonResult(ComparisonResult r);
+
     protected void evalInputs(IFrameTupleReference tuple) throws AlgebricksException {
-        evalLeft.evaluate(tuple, outLeft);
-        evalRight.evaluate(tuple, outRight);
+        evalLeft.evaluate(tuple, argLeft);
+        evalRight.evaluate(tuple, argRight);
+
+        argLeft.getValue(outLeft);
+        argRight.getValue(outRight);
     }
 
     // checks whether we can apply >, >=, <, and <= operations to the given type since
     // these operations can not be defined for certain types.
     protected void checkTotallyOrderable() throws AlgebricksException {
-        if (outLeft.getLength() != 0) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
-                    .deserialize(outLeft.getByteArray()[outLeft.getStartOffset()]);
+        if (argLeft.getLength() != 0) {
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
             switch (typeTag) {
                 case DURATION:
                 case INTERVAL:
@@ -142,8 +157,7 @@ public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
     // checks whether two types are comparable
     protected ComparableResultCode comparabilityCheck() {
         // just check TypeTags
-        return ABinaryComparator.isComparable(outLeft.getByteArray(), outLeft.getStartOffset(), 1,
-                outRight.getByteArray(), outRight.getStartOffset(), 1);
+        return ABinaryComparator.isComparable(argLeft.getTag(), argRight.getTag());
     }
 
     protected ComparisonResult compareResults() throws AlgebricksException {
@@ -152,15 +166,10 @@ public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
         ATypeTag typeTag1 = null;
         ATypeTag typeTag2 = null;
 
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftStartOffset = outLeft.getStartOffset();
-        byte[] rightBytes = outRight.getByteArray();
-        int rightStartOffset = outRight.getStartOffset();
-
         if (outLeft.getLength() == 0) {
             isLeftNull = true;
         } else {
-            typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStartOffset]);
+            typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
             if (typeTag1 == ATypeTag.NULL) {
                 isLeftNull = true;
             }
@@ -168,7 +177,7 @@ public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
         if (outRight.getLength() == 0) {
             isRightNull = true;
         } else {
-            typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(rightBytes[rightStartOffset]);
+            typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
             if (typeTag2 == ATypeTag.NULL) {
                 isRightNull = true;
             }
@@ -178,111 +187,7 @@ public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
             return ComparisonResult.UNKNOWN;
         }
 
-        switch (typeTag1) {
-            case INT8: {
-                return compareInt8WithArg(typeTag2);
-            }
-            case INT16: {
-                return compareInt16WithArg(typeTag2);
-            }
-            case INT32: {
-                return compareInt32WithArg(typeTag2);
-            }
-            case INT64: {
-                return compareInt64WithArg(typeTag2);
-            }
-            case FLOAT: {
-                return compareFloatWithArg(typeTag2);
-            }
-            case DOUBLE: {
-                return compareDoubleWithArg(typeTag2);
-            }
-            case STRING: {
-                return compareStringWithArg(typeTag2);
-            }
-            case BOOLEAN: {
-                return compareBooleanWithArg(typeTag2);
-            }
-
-            default: {
-                return compareStrongTypedWithArg(typeTag1, typeTag2);
-            }
-        }
-    }
-
-    private ComparisonResult compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag)
-            throws AlgebricksException {
-        if (expectedTypeTag != actualTypeTag) {
-            throw new AlgebricksException(
-                    "Comparison is undefined between " + expectedTypeTag + " and " + actualTypeTag + ".");
-        }
-        int result = 0;
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftOffset = outLeft.getStartOffset() + 1;
-        int leftLen = outLeft.getLength() - 1;
-        byte[] rightBytes = outRight.getByteArray();
-        int rightOffset = outRight.getStartOffset() + 1;
-        int rightLen = outRight.getLength() - 1;
-
-        try {
-            switch (actualTypeTag) {
-                case YEARMONTHDURATION:
-                case TIME:
-                case DATE:
-                    result = Integer.compare(AInt32SerializerDeserializer.getInt(leftBytes, leftOffset),
-                            AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
-                    break;
-                case DAYTIMEDURATION:
-                case DATETIME:
-                    result = Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftOffset),
-                            AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
-                    break;
-                case CIRCLE:
-                    result = circleBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case LINE:
-                    result = lineBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case POINT:
-                    result = pointBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case POINT3D:
-                    result = point3DBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case POLYGON:
-                    result = polygonBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case DURATION:
-                    result = durationBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case INTERVAL:
-                    result = intervalBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case RECTANGLE:
-                    result = rectangleBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case BINARY:
-                    result = byteArrayComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                case UUID:
-                    result = uuidBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
-                            rightLen);
-                    break;
-                default:
-                    throw new AlgebricksException("Comparison for " + actualTypeTag + " is not supported.");
-            }
-        } catch (HyracksDataException e) {
-            throw new AlgebricksException(e);
-        }
+        int result = ch.compare(typeTag1, typeTag2, outLeft, outRight);
         if (result == 0) {
             return ComparisonResult.EQUAL;
         } else if (result < 0) {
@@ -292,322 +197,4 @@ public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
         }
     }
 
-    private ComparisonResult compareBooleanWithArg(ATypeTag typeTag2) throws AlgebricksException {
-        if (typeTag2 == ATypeTag.BOOLEAN) {
-            byte b0 = outLeft.getByteArray()[outLeft.getStartOffset() + 1];
-            byte b1 = outRight.getByteArray()[outRight.getStartOffset() + 1];
-            return compareByte(b0, b1);
-        }
-        throw new AlgebricksException("Comparison is undefined between types ABoolean and " + typeTag2 + " .");
-    }
-
-    private ComparisonResult compareStringWithArg(ATypeTag typeTag2) throws AlgebricksException {
-        if (typeTag2 == ATypeTag.STRING) {
-            int result;
-            try {
-                result = strBinaryComp.compare(outLeft.getByteArray(), outLeft.getStartOffset() + 1,
-                        outLeft.getLength() - 1, outRight.getByteArray(), outRight.getStartOffset() + 1,
-                        outRight.getLength() - 1);
-            } catch (HyracksDataException e) {
-                throw new AlgebricksException(e);
-            }
-            if (result == 0) {
-                return ComparisonResult.EQUAL;
-            } else if (result < 0) {
-                return ComparisonResult.LESS_THAN;
-            } else {
-                return ComparisonResult.GREATER_THAN;
-            }
-        }
-        throw new AlgebricksException("Comparison is undefined between types AString and " + typeTag2 + " .");
-    }
-
-    private ComparisonResult compareDoubleWithArg(ATypeTag typeTag2) throws AlgebricksException {
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftOffset = outLeft.getStartOffset() + 1;
-        byte[] rightBytes = outRight.getByteArray();
-        int rightOffset = outRight.getStartOffset() + 1;
-
-        double s = ADoubleSerializerDeserializer.getDouble(leftBytes, leftOffset);
-        switch (typeTag2) {
-            case INT8: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            case INT16: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            case INT32: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            case INT64: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new AlgebricksException("Comparison is undefined between types ADouble and " + typeTag2 + " .");
-            }
-        }
-    }
-
-    private ComparisonResult compareFloatWithArg(ATypeTag typeTag2) throws AlgebricksException {
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftOffset = outLeft.getStartOffset() + 1;
-        byte[] rightBytes = outRight.getByteArray();
-        int rightOffset = outRight.getStartOffset() + 1;
-
-        float s = FloatPointable.getFloat(leftBytes, leftOffset);
-        switch (typeTag2) {
-            case INT8: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case INT16: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case INT32: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case INT64: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new AlgebricksException("Comparison is undefined between types AFloat and " + typeTag2 + " .");
-            }
-        }
-    }
-
-    private ComparisonResult compareInt64WithArg(ATypeTag typeTag2) throws AlgebricksException {
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftOffset = outLeft.getStartOffset() + 1;
-        byte[] rightBytes = outRight.getByteArray();
-        int rightOffset = outRight.getStartOffset() + 1;
-
-        long s = AInt64SerializerDeserializer.getLong(leftBytes, leftOffset);
-        switch (typeTag2) {
-            case INT8: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case INT16: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case INT32: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case INT64: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new AlgebricksException("Comparison is undefined between types AInt64 and " + typeTag2 + " .");
-            }
-        }
-    }
-
-    private ComparisonResult compareInt32WithArg(ATypeTag typeTag2) throws AlgebricksException {
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftOffset = outLeft.getStartOffset() + 1;
-        byte[] rightBytes = outRight.getByteArray();
-        int rightOffset = outRight.getStartOffset() + 1;
-
-        int s = IntegerPointable.getInteger(leftBytes, leftOffset);
-        switch (typeTag2) {
-            case INT8: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case INT16: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case INT32: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case INT64: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new AlgebricksException("Comparison is undefined between types AInt32 and " + typeTag2 + " .");
-            }
-        }
-    }
-
-    private ComparisonResult compareInt16WithArg(ATypeTag typeTag2) throws AlgebricksException {
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftOffset = outLeft.getStartOffset() + 1;
-        byte[] rightBytes = outRight.getByteArray();
-        int rightOffset = outRight.getStartOffset() + 1;
-
-        short s = AInt16SerializerDeserializer.getShort(leftBytes, leftOffset);
-        switch (typeTag2) {
-            case INT8: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
-                return compareShort(s, v2);
-            }
-            case INT16: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
-                return compareShort(s, v2);
-            }
-            case INT32: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
-                return compareInt(s, v2);
-            }
-            case INT64: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new AlgebricksException("Comparison is undefined between types AInt16 and " + typeTag2 + " .");
-            }
-        }
-    }
-
-    private ComparisonResult compareInt8WithArg(ATypeTag typeTag2) throws AlgebricksException {
-        byte[] leftBytes = outLeft.getByteArray();
-        int leftStart = outLeft.getStartOffset() + 1;
-        byte[] rightBytes = outRight.getByteArray();
-        int rightStart = outRight.getStartOffset() + 1;
-
-        byte s = AInt8SerializerDeserializer.getByte(leftBytes, leftStart);
-        switch (typeTag2) {
-            case INT8: {
-                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightStart);
-                return compareByte(s, v2);
-            }
-            case INT16: {
-                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightStart);
-                return compareShort(s, v2);
-            }
-            case INT32: {
-                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightStart);
-                return compareInt(s, v2);
-            }
-            case INT64: {
-                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightStart);
-                return compareLong(s, v2);
-            }
-            case FLOAT: {
-                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightStart);
-                return compareFloat(s, v2);
-            }
-            case DOUBLE: {
-                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightStart);
-                return compareDouble(s, v2);
-            }
-            default: {
-                throw new AlgebricksException("Comparison is undefined between types AInt16 and " + typeTag2 + " .");
-            }
-        }
-    }
-
-    private final ComparisonResult compareByte(int v1, int v2) {
-        if (v1 == v2) {
-            return ComparisonResult.EQUAL;
-        } else if (v1 < v2) {
-            return ComparisonResult.LESS_THAN;
-        } else {
-            return ComparisonResult.GREATER_THAN;
-        }
-    }
-
-    private final ComparisonResult compareShort(int v1, int v2) {
-        if (v1 == v2) {
-            return ComparisonResult.EQUAL;
-        } else if (v1 < v2) {
-            return ComparisonResult.LESS_THAN;
-        } else {
-            return ComparisonResult.GREATER_THAN;
-        }
-    }
-
-    private final ComparisonResult compareInt(int v1, int v2) {
-        if (v1 == v2) {
-            return ComparisonResult.EQUAL;
-        } else if (v1 < v2) {
-            return ComparisonResult.LESS_THAN;
-        } else {
-            return ComparisonResult.GREATER_THAN;
-        }
-    }
-
-    private final ComparisonResult compareLong(long v1, long v2) {
-        if (v1 == v2) {
-            return ComparisonResult.EQUAL;
-        } else if (v1 < v2) {
-            return ComparisonResult.LESS_THAN;
-        } else {
-            return ComparisonResult.GREATER_THAN;
-        }
-    }
-
-    private final ComparisonResult compareFloat(float v1, float v2) {
-        if (v1 == v2) {
-            return ComparisonResult.EQUAL;
-        } else if (v1 < v2) {
-            return ComparisonResult.LESS_THAN;
-        } else {
-            return ComparisonResult.GREATER_THAN;
-        }
-    }
-
-    private final ComparisonResult compareDouble(double v1, double v2) {
-        if (v1 == v2) {
-            return ComparisonResult.EQUAL;
-        } else if (v1 < v2) {
-            return ComparisonResult.LESS_THAN;
-        } else {
-            return ComparisonResult.GREATER_THAN;
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f29b26fd/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java
index e549345..649f262 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java
@@ -18,16 +18,11 @@
  */
 package org.apache.asterix.runtime.evaluators.comparisons;
 
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class ComparisonEvalFactory implements IScalarEvaluatorFactory {
     private static final long serialVersionUID = 1L;
@@ -78,46 +73,14 @@ public class ComparisonEvalFactory implements IScalarEvaluatorFactory {
         }
 
         @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            evalInputs(tuple);
-            // Checks whether two types are comparable
-            switch (comparabilityCheck()) {
-                case UNKNOWN:
-                    // result:UNKNOWN - NULL value found
-                    try {
-                        nullSerde.serialize(ANull.NULL, out);
-                        result.set(resultStorage);
-                        return;
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                case FALSE:
-                    // result:FALSE - two types cannot be compared. Thus we return FALSE since this is equality comparison
-                    ABoolean b = ABoolean.FALSE;
-                    try {
-                        serde.serialize(b, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                case TRUE:
-                    // Two types can be compared
-                    ComparisonResult r = compareResults();
-                    ABoolean b1 = (r == ComparisonResult.EQUAL) ? ABoolean.TRUE : ABoolean.FALSE;
-                    try {
-                        serde.serialize(b1, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                default:
-                    throw new AlgebricksException(
-                            "Equality Comparison cannot be processed. The return code from ComparabilityCheck is not correct.");
-            }
-            result.set(resultStorage);
+        protected boolean getComparisonResult(ComparisonResult r) {
+            return (r == ComparisonResult.EQUAL);
         }
 
+        @Override
+        protected boolean isTotallyOrderable() {
+            return false;
+        }
     }
 
     static class InequalityComparisonEvaluator extends AbstractComparisonEvaluator {
@@ -127,47 +90,14 @@ public class ComparisonEvalFactory implements IScalarEvaluatorFactory {
         }
 
         @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            evalInputs(tuple);
-
-            // Checks whether two types are comparable
-            switch (comparabilityCheck()) {
-                case UNKNOWN:
-                    // result:UNKNOWN - NULL value found
-                    try {
-                        nullSerde.serialize(ANull.NULL, out);
-                        result.set(resultStorage);
-                        return;
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                case FALSE:
-                    // result:FALSE - two types cannot be compared. Thus we return TRUE since this is NOT EQ comparison.
-                    ABoolean b = ABoolean.TRUE;
-                    try {
-                        serde.serialize(b, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                case TRUE:
-                    // Two types can be compared
-                    ComparisonResult r = compareResults();
-                    ABoolean b1 = (r != ComparisonResult.EQUAL) ? ABoolean.TRUE : ABoolean.FALSE;
-                    try {
-                        serde.serialize(b1, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                default:
-                    throw new AlgebricksException(
-                            "Inequality Comparison cannot be processed. The return code from ComparabilityCheck is not correct.");
-            }
-            result.set(resultStorage);
+        protected boolean getComparisonResult(ComparisonResult r) {
+            return (r != ComparisonResult.EQUAL);
         }
 
+        @Override
+        protected boolean isTotallyOrderable() {
+            return false;
+        }
     }
 
     static class GreaterThanOrEqualComparisonEvaluator extends AbstractComparisonEvaluator {
@@ -177,52 +107,14 @@ public class ComparisonEvalFactory implements IScalarEvaluatorFactory {
         }
 
         @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            evalInputs(tuple);
-
-            // checks whether we can apply >, >=, <, and <= to the given type since
-            // these operations cannot be defined for certain types.
-            checkTotallyOrderable();
-
-            // Checks whether two types are comparable
-            switch (comparabilityCheck()) {
-                case UNKNOWN:
-                    // result:UNKNOWN - NULL value found
-                    try {
-                        nullSerde.serialize(ANull.NULL, out);
-                        result.set(resultStorage);
-                        return;
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                case FALSE:
-                    // result:FALSE - two types cannot be compared. Thus we return FALSE since this is an inequality comparison.
-                    ABoolean b = ABoolean.FALSE;
-                    try {
-                        serde.serialize(b, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                case TRUE:
-                    // Two types can be compared
-                    ComparisonResult r = compareResults();
-                    ABoolean b1 = (r == ComparisonResult.EQUAL || r == ComparisonResult.GREATER_THAN) ? ABoolean.TRUE
-                            : ABoolean.FALSE;
-                    try {
-                        serde.serialize(b1, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                default:
-                    throw new AlgebricksException(
-                            "Inequality Comparison cannot be processed. The return code from ComparabilityCheck is not correct.");
-            }
-            result.set(resultStorage);
+        protected boolean getComparisonResult(ComparisonResult r) {
+            return (r == ComparisonResult.EQUAL || r == ComparisonResult.GREATER_THAN);
         }
 
+        @Override
+        protected boolean isTotallyOrderable() {
+            return true;
+        }
     }
 
     static class GreaterThanComparisonEvaluator extends AbstractComparisonEvaluator {
@@ -232,51 +124,14 @@ public class ComparisonEvalFactory implements IScalarEvaluatorFactory {
         }
 
         @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            evalInputs(tuple);
-
-            // checks whether we can apply >, >=, <, and <= to the given type since
-            // these operations cannot be defined for certain types.
-            checkTotallyOrderable();
-
-            // Checks whether two types are comparable
-            switch (comparabilityCheck()) {
-                case UNKNOWN:
-                    // result:UNKNOWN - NULL value found
-                    try {
-                        nullSerde.serialize(ANull.NULL, out);
-                        result.set(resultStorage);
-                        return;
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                case FALSE:
-                    // result:FALSE - two types cannot be compared. Thus we return FALSE since this is an inequality comparison.
-                    ABoolean b = ABoolean.FALSE;
-                    try {
-                        serde.serialize(b, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                case TRUE:
-                    // Two types can be compared
-                    ComparisonResult r = compareResults();
-                    ABoolean b1 = (r == ComparisonResult.GREATER_THAN) ? ABoolean.TRUE : ABoolean.FALSE;
-                    try {
-                        serde.serialize(b1, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                default:
-                    throw new AlgebricksException(
-                            "Inequality Comparison cannot be processed. The return code from ComparabilityCheck is not correct.");
-            }
-            result.set(resultStorage);
+        protected boolean getComparisonResult(ComparisonResult r) {
+            return (r == ComparisonResult.GREATER_THAN);
         }
 
+        @Override
+        protected boolean isTotallyOrderable() {
+            return true;
+        }
     }
 
     static class LessThanOrEqualComparisonEvaluator extends AbstractComparisonEvaluator {
@@ -286,52 +141,14 @@ public class ComparisonEvalFactory implements IScalarEvaluatorFactory {
         }
 
         @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            evalInputs(tuple);
-
-            // checks whether we can apply >, >=, <, and <= to the given type since
-            // these operations cannot be defined for certain types.
-            checkTotallyOrderable();
-
-            // Checks whether two types are comparable
-            switch (comparabilityCheck()) {
-                case UNKNOWN:
-                    // result:UNKNOWN - NULL value found
-                    try {
-                        nullSerde.serialize(ANull.NULL, out);
-                        result.set(resultStorage);
-                        return;
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                case FALSE:
-                    // result:FALSE - two types cannot be compared. Thus we return FALSE since this is an inequality comparison.
-                    ABoolean b = ABoolean.FALSE;
-                    try {
-                        serde.serialize(b, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                case TRUE:
-                    // Two types can be compared
-                    ComparisonResult r = compareResults();
-                    ABoolean b1 = (r == ComparisonResult.EQUAL || r == ComparisonResult.LESS_THAN) ? ABoolean.TRUE
-                            : ABoolean.FALSE;
-                    try {
-                        serde.serialize(b1, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                default:
-                    throw new AlgebricksException(
-                            "Inequality Comparison cannot be processed. The return code from ComparabilityCheck is not correct.");
-            }
-            result.set(resultStorage);
+        protected boolean getComparisonResult(ComparisonResult r) {
+            return (r == ComparisonResult.EQUAL || r == ComparisonResult.LESS_THAN);
         }
 
+        @Override
+        protected boolean isTotallyOrderable() {
+            return true;
+        }
     }
 
     static class LessThanComparisonEvaluator extends AbstractComparisonEvaluator {
@@ -341,51 +158,14 @@ public class ComparisonEvalFactory implements IScalarEvaluatorFactory {
         }
 
         @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
-            resultStorage.reset();
-            evalInputs(tuple);
-
-            // checks whether we can apply >, >=, <, and <= to the given type since
-            // these operations cannot be defined for certain types.
-            checkTotallyOrderable();
-
-            // Checks whether two types are comparable
-            switch (comparabilityCheck()) {
-                case UNKNOWN:
-                    // result:UNKNOWN - NULL value found
-                    try {
-                        nullSerde.serialize(ANull.NULL, out);
-                        result.set(resultStorage);
-                        return;
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                case FALSE:
-                    // result:FALSE - two types cannot be compared. Thus we return FALSE since this is an inequality comparison.
-                    ABoolean b = ABoolean.FALSE;
-                    try {
-                        serde.serialize(b, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                case TRUE:
-                    // Two types can be compared
-                    ComparisonResult r = compareResults();
-                    ABoolean b1 = (r == ComparisonResult.LESS_THAN) ? ABoolean.TRUE : ABoolean.FALSE;
-                    try {
-                        serde.serialize(b1, out);
-                    } catch (HyracksDataException e) {
-                        throw new AlgebricksException(e);
-                    }
-                    break;
-                default:
-                    throw new AlgebricksException(
-                            "Inequality Comparison cannot be processed. The return code from ComparabilityCheck is not correct.");
-            }
-            result.set(resultStorage);
+        protected boolean getComparisonResult(ComparisonResult r) {
+            return (r == ComparisonResult.LESS_THAN);
         }
 
+        @Override
+        protected boolean isTotallyOrderable() {
+            return true;
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f29b26fd/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f2be58f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
@@ -0,0 +1,486 @@
+/*
+ * 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.asterix.runtime.evaluators.comparisons;
+
+import java.io.Serializable;
+
+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;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ALinePartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.APoint3DPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.APointPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.APolygonPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ARectanglePartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AUUIDPartialBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+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;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
+import org.apache.hyracks.data.std.primitive.FloatPointable;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+
+public class ComparisonHelper implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final IBinaryComparator strBinaryComp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator circleBinaryComp = ACirclePartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator durationBinaryComp = ADurationPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator intervalBinaryComp = AIntervalAscPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator lineBinaryComparator = ALinePartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator pointBinaryComparator = APointPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator point3DBinaryComparator = APoint3DPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator polygonBinaryComparator = APolygonPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator rectangleBinaryComparator = ARectanglePartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator uuidBinaryComparator = AUUIDPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    private final IBinaryComparator byteArrayComparator = new PointableBinaryComparatorFactory(
+            ByteArrayPointable.FACTORY).createBinaryComparator();
+
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2)
+            throws AlgebricksException {
+        switch (typeTag1) {
+            case INT8:
+                return compareInt8WithArg(typeTag2, arg1, arg2);
+            case INT16:
+                return compareInt16WithArg(typeTag2, arg1, arg2);
+            case INT32:
+                return compareInt32WithArg(typeTag2, arg1, arg2);
+            case INT64:
+                return compareInt64WithArg(typeTag2, arg1, arg2);
+            case FLOAT:
+                return compareFloatWithArg(typeTag2, arg1, arg2);
+            case DOUBLE:
+                return compareDoubleWithArg(typeTag2, arg1, arg2);
+            case STRING:
+                return compareStringWithArg(typeTag2, arg1, arg2);
+            case BOOLEAN:
+                return compareBooleanWithArg(typeTag2, arg1, arg2);
+            default:
+                return compareStrongTypedWithArg(typeTag1, typeTag2, arg1, arg2);
+        }
+    }
+
+    private int compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag, IPointable arg1,
+            IPointable arg2) throws AlgebricksException {
+        if (expectedTypeTag != actualTypeTag) {
+            throw new AlgebricksException(
+                    "Comparison is undefined between " + expectedTypeTag + " and " + actualTypeTag + ".");
+        }
+        int result = 0;
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        int leftLen = arg1.getLength() - 1;
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+        int rightLen = arg2.getLength() - 1;
+
+        try {
+            switch (actualTypeTag) {
+                case YEARMONTHDURATION:
+                case TIME:
+                case DATE:
+                    result = Integer.compare(AInt32SerializerDeserializer.getInt(leftBytes, leftOffset),
+                            AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+                    break;
+                case DAYTIMEDURATION:
+                case DATETIME:
+                    result = Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftOffset),
+                            AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+                    break;
+                case CIRCLE:
+                    result = circleBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case LINE:
+                    result = lineBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case POINT:
+                    result = pointBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case POINT3D:
+                    result = point3DBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case POLYGON:
+                    result = polygonBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case DURATION:
+                    result = durationBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case INTERVAL:
+                    result = intervalBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case RECTANGLE:
+                    result = rectangleBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case BINARY:
+                    result = byteArrayComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                case UUID:
+                    result = uuidBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                            rightLen);
+                    break;
+                default:
+                    throw new AlgebricksException("Comparison for " + actualTypeTag + " is not supported.");
+            }
+        } catch (HyracksDataException e) {
+            throw new AlgebricksException(e);
+        }
+        return result;
+    }
+
+    private int compareBooleanWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        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 + " .");
+    }
+
+    private int compareStringWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        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);
+            }
+            return result;
+        }
+        throw new AlgebricksException("Comparison is undefined between types AString and " + typeTag2 + " .");
+    }
+
+    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        double s = ADoubleSerializerDeserializer.getDouble(leftBytes, leftOffset);
+        switch (typeTag2) {
+            case INT8: {
+                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            case INT16: {
+                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            case INT32: {
+                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            case INT64: {
+                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            case FLOAT: {
+                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            case DOUBLE: {
+                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            default: {
+                throw new AlgebricksException("Comparison is undefined between types ADouble and " + typeTag2 + " .");
+            }
+        }
+    }
+
+    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        float s = FloatPointable.getFloat(leftBytes, leftOffset);
+        switch (typeTag2) {
+            case INT8: {
+                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case INT16: {
+                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case INT32: {
+                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case INT64: {
+                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case FLOAT: {
+                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case DOUBLE: {
+                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            default: {
+                throw new AlgebricksException("Comparison is undefined between types AFloat and " + typeTag2 + " .");
+            }
+        }
+    }
+
+    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        long s = AInt64SerializerDeserializer.getLong(leftBytes, leftOffset);
+        switch (typeTag2) {
+            case INT8: {
+                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
+                return compareLong(s, v2);
+            }
+            case INT16: {
+                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
+                return compareLong(s, v2);
+            }
+            case INT32: {
+                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
+                return compareLong(s, v2);
+            }
+            case INT64: {
+                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
+                return compareLong(s, v2);
+            }
+            case FLOAT: {
+                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case DOUBLE: {
+                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            default: {
+                throw new AlgebricksException("Comparison is undefined between types AInt64 and " + typeTag2 + " .");
+            }
+        }
+    }
+
+    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        int s = IntegerPointable.getInteger(leftBytes, leftOffset);
+        switch (typeTag2) {
+            case INT8: {
+                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
+                return compareInt(s, v2);
+            }
+            case INT16: {
+                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
+                return compareInt(s, v2);
+            }
+            case INT32: {
+                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
+                return compareInt(s, v2);
+            }
+            case INT64: {
+                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
+                return compareLong(s, v2);
+            }
+            case FLOAT: {
+                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case DOUBLE: {
+                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            default: {
+                throw new AlgebricksException("Comparison is undefined between types AInt32 and " + typeTag2 + " .");
+            }
+        }
+    }
+
+    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        short s = AInt16SerializerDeserializer.getShort(leftBytes, leftOffset);
+        switch (typeTag2) {
+            case INT8: {
+                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightOffset);
+                return compareShort(s, v2);
+            }
+            case INT16: {
+                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightOffset);
+                return compareShort(s, v2);
+            }
+            case INT32: {
+                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightOffset);
+                return compareInt(s, v2);
+            }
+            case INT64: {
+                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightOffset);
+                return compareLong(s, v2);
+            }
+            case FLOAT: {
+                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightOffset);
+                return compareFloat(s, v2);
+            }
+            case DOUBLE: {
+                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightOffset);
+                return compareDouble(s, v2);
+            }
+            default: {
+                throw new AlgebricksException("Comparison is undefined between types AInt16 and " + typeTag2 + " .");
+            }
+        }
+    }
+
+    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws AlgebricksException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftStart = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightStart = arg2.getStartOffset();
+
+        byte s = AInt8SerializerDeserializer.getByte(leftBytes, leftStart);
+        switch (typeTag2) {
+            case INT8: {
+                byte v2 = AInt8SerializerDeserializer.getByte(rightBytes, rightStart);
+                return compareByte(s, v2);
+            }
+            case INT16: {
+                short v2 = AInt16SerializerDeserializer.getShort(rightBytes, rightStart);
+                return compareShort(s, v2);
+            }
+            case INT32: {
+                int v2 = AInt32SerializerDeserializer.getInt(rightBytes, rightStart);
+                return compareInt(s, v2);
+            }
+            case INT64: {
+                long v2 = AInt64SerializerDeserializer.getLong(rightBytes, rightStart);
+                return compareLong(s, v2);
+            }
+            case FLOAT: {
+                float v2 = AFloatSerializerDeserializer.getFloat(rightBytes, rightStart);
+                return compareFloat(s, v2);
+            }
+            case DOUBLE: {
+                double v2 = ADoubleSerializerDeserializer.getDouble(rightBytes, rightStart);
+                return compareDouble(s, v2);
+            }
+            default: {
+                throw new AlgebricksException("Comparison is undefined between types AInt16 and " + typeTag2 + " .");
+            }
+        }
+    }
+
+    private final int compareByte(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareShort(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareInt(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareLong(long v1, long v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareFloat(float v1, float v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareDouble(double v1, double v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/f29b26fd/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
index e48e5b6..1cd3919 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/AbstractIntervalLogicFuncDescriptor.java
@@ -20,10 +20,10 @@ package org.apache.asterix.runtime.evaluators.functions.temporal;
 
 import java.io.DataOutput;
 
-import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
@@ -35,7 +35,7 @@ import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
@@ -57,10 +57,17 @@ public abstract class AbstractIntervalLogicFuncDescriptor extends AbstractScalar
 
                 return new IScalarEvaluator() {
 
+                    protected final IntervalLogic il = new IntervalLogic();
                     private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
                     private DataOutput out = resultStorage.getDataOutput();
-                    private IPointable argPtr0 = new VoidPointable();
-                    private IPointable argPtr1 = new VoidPointable();
+                    private TaggedValuePointable argPtr0 = (TaggedValuePointable) TaggedValuePointable.FACTORY
+                            .createPointable();
+                    private TaggedValuePointable argPtr1 = (TaggedValuePointable) TaggedValuePointable.FACTORY
+                            .createPointable();
+                    private AIntervalPointable interval0 = (AIntervalPointable) AIntervalPointable.FACTORY
+                            .createPointable();
+                    private AIntervalPointable interval1 = (AIntervalPointable) AIntervalPointable.FACTORY
+                            .createPointable();
                     private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
                     private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
 
@@ -78,40 +85,31 @@ public abstract class AbstractIntervalLogicFuncDescriptor extends AbstractScalar
                         eval0.evaluate(tuple, argPtr0);
                         eval1.evaluate(tuple, argPtr1);
 
-                        byte[] bytes0 = argPtr0.getByteArray();
-                        int offset0 = argPtr0.getStartOffset();
-                        byte[] bytes1 = argPtr1.getByteArray();
-                        int offset1 = argPtr1.getStartOffset();
-
                         try {
-                            if (bytes0[offset0] == ATypeTag.SERIALIZED_NULL_TYPE_TAG
-                                    || bytes1[offset1] == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
+                            if (argPtr0.getTag() == ATypeTag.SERIALIZED_NULL_TYPE_TAG
+                                    || argPtr1.getTag() == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
                                 nullSerde.serialize(ANull.NULL, out);
                                 result.set(resultStorage);
                                 return;
                             }
 
-                            if (bytes0[offset0] != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG
-                                    || bytes1[offset1] != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
+                            if (argPtr0.getTag() != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG
+                                    || argPtr1.getTag() != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
                                 throw new AlgebricksException(getIdentifier().getName()
                                         + ": expects input type (INTERVAL, INTERVAL) but got ("
-                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]) + ", "
-                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]) + ")");
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr0.getTag()) + ", "
+                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argPtr1.getTag()) + ")");
                             }
 
-                            if (AIntervalSerializerDeserializer.getIntervalTimeType(bytes0,
-                                    offset0 + 1) != AIntervalSerializerDeserializer.getIntervalTimeType(bytes1,
-                                            offset1 + 1)) {
+                            argPtr0.getValue(interval0);
+                            argPtr1.getValue(interval1);
+
+                            if (interval0.getType() != interval1.getType()) {
                                 throw new AlgebricksException(getIdentifier().getName()
                                         + ": failed to compare intervals with different internal time type.");
                             }
 
-                            ABoolean res = (compareIntervals(
-                                    AIntervalSerializerDeserializer.getIntervalStart(bytes0, offset0 + 1),
-                                    AIntervalSerializerDeserializer.getIntervalEnd(bytes0, offset0 + 1),
-                                    AIntervalSerializerDeserializer.getIntervalStart(bytes1, offset1 + 1),
-                                    AIntervalSerializerDeserializer.getIntervalEnd(bytes1, offset1 + 1)))
-                                            ? ABoolean.TRUE : ABoolean.FALSE;
+                            ABoolean res = (compareIntervals(il, interval0, interval1)) ? ABoolean.TRUE : ABoolean.FALSE;
 
                             booleanSerde.serialize(res, out);
                         } catch (HyracksDataException hex) {
@@ -124,6 +122,7 @@ public abstract class AbstractIntervalLogicFuncDescriptor extends AbstractScalar
         };
     }
 
-    protected abstract boolean compareIntervals(long s1, long e1, long s2, long e2);
+    protected abstract boolean compareIntervals(IntervalLogic il, AIntervalPointable ip1, AIntervalPointable ip2)
+            throws AlgebricksException;
 
 }