You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org> on 2019/02/04 08:19:13 UTC

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Ali Alsuliman has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................

[ASTERIXDB-2516][RT] add support for deep comparison 1

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

Details:
add support for deep comparison

Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
---
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
27 files changed, 995 insertions(+), 172 deletions(-)


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

diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
index e3b97f5..e3846b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
@@ -1 +1 @@
-[  ]
+{ "circle0": null, "circle1": null, "circle2": null, "circle3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
index e3b97f5..60e80d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
@@ -1 +1 @@
-[  ]
+{ "duration0": null, "duration1": null, "duration2": null, "duration3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
index e3b97f5..76c2bb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
@@ -1 +1 @@
-[  ]
+{ "interval0": null, "interval1": null, "interval2": null, "interval3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
index e3b97f5..31cf931 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
@@ -1 +1 @@
-[  ]
+{ "line0": null, "line1": null, "line2": null, "line3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
index e3b97f5..3e24175 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
@@ -1 +1 @@
-[  ]
+{ "point0": null, "point1": null, "point2": null, "point3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
index e3b97f5..8a0655b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
@@ -1 +1 @@
-[  ]
+{ "polygon0": null, "polygon1": null, "polygon2": null, "polygon3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
index e3b97f5..5429dcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
@@ -1 +1 @@
-[  ]
+{ "rectangle0": null, "rectangle1": null, "rectangle2": null, "rectangle3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index ceee5f9..d7ca047 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -2303,43 +2303,36 @@
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_duration">
         <output-dir compare="Text">issue363_inequality_duration</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type duration</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_interval">
         <output-dir compare="Text">issue363_inequality_interval</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type interval</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_point">
         <output-dir compare="Text">issue363_inequality_point</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type point</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_line">
         <output-dir compare="Text">issue363_inequality_line</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type line</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_polygon">
         <output-dir compare="Text">issue363_inequality_polygon</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type polygon</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_rectangle">
         <output-dir compare="Text">issue363_inequality_rectangle</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type rectangle</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_circle">
         <output-dir compare="Text">issue363_inequality_circle</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type circle</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
new file mode 100644
index 0000000..7f9769a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
@@ -0,0 +1,57 @@
+/*
+ * 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.dataflow.data.common;
+
+import org.apache.asterix.om.base.IAObject;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILogicalBinaryComparator {
+    enum Result {
+        MISSING,
+        NULL,
+        EQ,
+        LT,
+        GT,
+        MISMATCH
+    }
+
+    enum Operation {
+        EQUALITY,
+        INEQUALITY
+    }
+
+    static Result returnResult(int result) {
+        if (result == 0) {
+            return Result.EQ;
+        } else if (result < 0) {
+            return Result.LT;
+        } else {
+            return Result.GT;
+        }
+    }
+
+    Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen,
+            Operation operation) throws HyracksDataException;
+
+    Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant, Operation operation);
+
+    Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen, Operation operation);
+
+    Result compare(IAObject leftConstant, IAObject rightConstant, Operation operation);
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java
new file mode 100644
index 0000000..5f20d3c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java
@@ -0,0 +1,171 @@
+/*
+ * 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.dataflow.data.common;
+
+import static org.apache.asterix.om.types.ATypeTag.BIGINT;
+import static org.apache.asterix.om.types.ATypeTag.DOUBLE;
+import static org.apache.asterix.om.types.ATypeTag.FLOAT;
+import static org.apache.asterix.om.types.ATypeTag.INTEGER;
+import static org.apache.asterix.om.types.ATypeTag.MISSING;
+import static org.apache.asterix.om.types.ATypeTag.NULL;
+import static org.apache.asterix.om.types.ATypeTag.SMALLINT;
+import static org.apache.asterix.om.types.ATypeTag.TINYINT;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.LogicalComplexBinaryComparator;
+import org.apache.asterix.dataflow.data.nontagged.comparators.LogicalGenericBinaryComparator;
+import org.apache.asterix.dataflow.data.nontagged.comparators.LogicalScalarBinaryComparator;
+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.om.base.ADouble;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+
+public class LogicalComparatorUtil {
+
+    public static ILogicalBinaryComparator createLogicalComparator(IAType left, IAType right) {
+        IAType leftType = TypeComputeUtils.getActualType(left);
+        IAType rightType = TypeComputeUtils.getActualType(right);
+
+        if (leftType.getTypeTag().isDerivedType() && rightType.getTypeTag().isDerivedType()) {
+            return new LogicalComplexBinaryComparator(leftType, rightType);
+        } else if (leftType.getTypeTag() == ATypeTag.ANY || rightType.getTypeTag() == ATypeTag.ANY) {
+            return new LogicalGenericBinaryComparator(leftType, rightType);
+        } else {
+            return new LogicalScalarBinaryComparator();
+        }
+    }
+
+    public static ILogicalBinaryComparator.Result returnMissingOrNull(ATypeTag leftTag, ATypeTag rightTag) {
+        if (leftTag == MISSING || rightTag == MISSING) {
+            return ILogicalBinaryComparator.Result.MISSING;
+        }
+        if (leftTag == NULL || rightTag == NULL) {
+            return ILogicalBinaryComparator.Result.NULL;
+        }
+        if (!ATypeHierarchy.isCompatible(leftTag, rightTag)) {
+            return ILogicalBinaryComparator.Result.MISMATCH;
+        }
+        return null;
+    }
+
+    // checking that left and right are compatible has to be done before calling this
+    public static ILogicalBinaryComparator.Result compareNumbers(ATypeTag leftTag, byte[] b1, int s1, ATypeTag rightTag,
+            byte[] b2, int s2) {
+        int result;
+        if (leftTag == DOUBLE || rightTag == DOUBLE) {
+            result = Double.compare(getValue(leftTag, b1, s1), getValue(rightTag, b2, s2));
+        } else if (leftTag == FLOAT || rightTag == FLOAT) {
+            result = Float.compare((float) getValue(leftTag, b1, s1), (float) getValue(rightTag, b2, s2));
+        } else if (leftTag == BIGINT || rightTag == BIGINT) {
+            result = Long.compare((long) getValue(leftTag, b1, s1), (long) getValue(rightTag, b2, s2));
+        } else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
+            result = Integer.compare((int) getValue(leftTag, b1, s1), (int) getValue(rightTag, b2, s2));
+        } else {
+            // two types that are compatible but are not numeric??
+            return null;
+        }
+        return ILogicalBinaryComparator.returnResult(result);
+    }
+
+    public static ILogicalBinaryComparator.Result compareNumWithConstant(ATypeTag leftTag, byte[] b1, int s1,
+            IAObject rightConstant) {
+        int result;
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag == DOUBLE || rightTag == DOUBLE) {
+            result = Double.compare(getValue(leftTag, b1, s1), getConstantValue(rightConstant));
+        } else if (leftTag == FLOAT || rightTag == FLOAT) {
+            result = Float.compare((float) getValue(leftTag, b1, s1), (float) getConstantValue(rightConstant));
+        } else if (leftTag == BIGINT || rightTag == BIGINT) {
+            result = Long.compare((long) getValue(leftTag, b1, s1), (long) getConstantValue(rightConstant));
+        } else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
+            result = Integer.compare((int) getValue(leftTag, b1, s1), (int) getConstantValue(rightConstant));
+        } else {
+            // two types that are compatible but are not numeric??
+            return null;
+        }
+        return ILogicalBinaryComparator.returnResult(result);
+    }
+
+    public static ILogicalBinaryComparator.Result compareConstants(IAObject leftConstant, IAObject rightConstant) {
+        int result;
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag == DOUBLE || rightTag == DOUBLE) {
+            result = Double.compare(getConstantValue(leftConstant), getConstantValue(rightConstant));
+        } else if (leftTag == FLOAT || rightTag == FLOAT) {
+            result = Float.compare((float) getConstantValue(leftConstant), (float) getConstantValue(rightConstant));
+        } else if (leftTag == BIGINT || rightTag == BIGINT) {
+            result = Long.compare((long) getConstantValue(leftConstant), (long) getConstantValue(rightConstant));
+        } else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
+            result = Integer.compare((int) getConstantValue(leftConstant), (int) getConstantValue(rightConstant));
+        } else {
+            // two types that are compatible but are not numeric??
+            return null;
+        }
+        return ILogicalBinaryComparator.returnResult(result);
+    }
+
+    private static double getValue(ATypeTag numericTag, byte[] b, int s) {
+        s++;
+        switch (numericTag) {
+            case TINYINT:
+                return AInt8SerializerDeserializer.getByte(b, s);
+            case SMALLINT:
+                return AInt16SerializerDeserializer.getShort(b, s);
+            case INTEGER:
+                return AInt32SerializerDeserializer.getInt(b, s);
+            case BIGINT:
+                return AInt64SerializerDeserializer.getLong(b, s);
+            case FLOAT:
+                return AFloatSerializerDeserializer.getFloat(b, s);
+            default:
+                return ADoubleSerializerDeserializer.getDouble(b, s);
+        }
+    }
+
+    private static double getConstantValue(IAObject numeric) {
+        IAType type = numeric.getType();
+        if (type == BuiltinType.ADOUBLE) {
+            return ((ADouble) numeric).getDoubleValue();
+        } else if (type == BuiltinType.AFLOAT) {
+            return ((AFloat) numeric).getFloatValue();
+        } else if (type == BuiltinType.AINT64) {
+            return ((AInt64) numeric).getLongValue();
+        } else if (type == BuiltinType.AINT32) {
+            return ((AInt32) numeric).getIntegerValue();
+        } else if (type == BuiltinType.AINT16) {
+            return ((AInt16) numeric).getShortValue();
+        } else {
+            return ((AInt8) numeric).getByteValue();
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
new file mode 100644
index 0000000..3170608
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
@@ -0,0 +1,98 @@
+/*
+ * 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.dataflow.data.nontagged.comparators;
+
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.dataflow.data.common.LogicalComparatorUtil;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LogicalComplexBinaryComparator implements ILogicalBinaryComparator {
+
+    private final IAType leftType;
+    private final IAType rightType;
+    private final LogicalScalarBinaryComparator scalarComparator = new LogicalScalarBinaryComparator();
+
+    public LogicalComplexBinaryComparator(IAType leftType, IAType rightType) {
+        this.leftType = leftType;
+        this.rightType = rightType;
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen,
+            Operation operation) throws HyracksDataException {
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(rightBytes[rightStart]);
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNull(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (!leftTag.isDerivedType() || !rightTag.isDerivedType()) {
+            return Result.NULL;
+        }
+        // TODO(ali): complex types(records, arrays, multisets) logic here
+        return Result.NULL;
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant, Operation operation) {
+        // TODO(ali): not defined currently for constant complex types
+        if (rightConstant == null) {
+            return Result.NULL;
+        }
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNull(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen, Operation operation) {
+        // TODO(ali): not defined currently for constant complex types
+        Result result = compare(rightBytes, rightStart, rightLen, leftConstant, operation);
+        if (result == Result.LT) {
+            return Result.GT;
+        } else if (result == Result.GT) {
+            return Result.LT;
+        }
+        return result;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, IAObject rightConstant, Operation operation) {
+        // TODO(ali): not defined currently for constant complex types
+        if (leftConstant == null || rightConstant == null) {
+            // illegal state maybe???
+            return Result.NULL;
+        }
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNull(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
new file mode 100644
index 0000000..e3852b9
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
@@ -0,0 +1,86 @@
+/*
+ * 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.dataflow.data.nontagged.comparators;
+
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LogicalGenericBinaryComparator implements ILogicalBinaryComparator {
+
+    private final LogicalComplexBinaryComparator complexComparator;
+    private final LogicalScalarBinaryComparator scalarComparator = new LogicalScalarBinaryComparator();
+
+    public LogicalGenericBinaryComparator(IAType leftType, IAType rightType) {
+        complexComparator = new LogicalComplexBinaryComparator(leftType, rightType);
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen,
+            Operation op) throws HyracksDataException {
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(rightBytes[rightStart]);
+        if (leftTag.isDerivedType() && rightTag.isDerivedType()) {
+            return complexComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen, op);
+        }
+        return scalarComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen, op);
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant, Operation operation) {
+        if (rightConstant == null) {
+            // illegal state maybe???
+            return Result.NULL;
+        }
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag.isDerivedType() && rightTag.isDerivedType()) {
+            return complexComparator.compare(leftBytes, leftStart, leftLen, rightConstant, operation);
+        }
+        return scalarComparator.compare(leftBytes, leftStart, leftLen, rightConstant, operation);
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen, Operation operation) {
+        Result result = compare(rightBytes, rightStart, rightLen, leftConstant, operation);
+        if (result == Result.LT) {
+            return Result.GT;
+        } else if (result == Result.GT) {
+            return Result.LT;
+        }
+        return result;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, IAObject rightConstant, Operation operation) {
+        if (leftConstant == null || rightConstant == null) {
+            // illegal state maybe???
+            return Result.NULL;
+        }
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag.isDerivedType() && rightTag.isDerivedType()) {
+            return complexComparator.compare(leftConstant, rightConstant, operation);
+        }
+        return scalarComparator.compare(leftConstant, rightConstant, operation);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
new file mode 100644
index 0000000..6387ecc
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
@@ -0,0 +1,221 @@
+/*
+ * 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.dataflow.data.nontagged.comparators;
+
+import static org.apache.asterix.om.types.ATypeTag.CIRCLE;
+import static org.apache.asterix.om.types.ATypeTag.DURATION;
+import static org.apache.asterix.om.types.ATypeTag.INTERVAL;
+import static org.apache.asterix.om.types.ATypeTag.LINE;
+import static org.apache.asterix.om.types.ATypeTag.POINT;
+import static org.apache.asterix.om.types.ATypeTag.POINT3D;
+import static org.apache.asterix.om.types.ATypeTag.POLYGON;
+import static org.apache.asterix.om.types.ATypeTag.RECTANGLE;
+
+import java.util.EnumSet;
+
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.dataflow.data.common.LogicalComparatorUtil;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+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.primitive.ByteArrayPointable;
+
+public class LogicalScalarBinaryComparator implements ILogicalBinaryComparator {
+
+    private final IBinaryComparator strBinaryComp =
+            BinaryComparatorFactoryProvider.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();
+
+    private static final EnumSet<ATypeTag> UNDEFINED_TYPES =
+            EnumSet.of(DURATION, INTERVAL, LINE, POINT, POINT3D, POLYGON, CIRCLE, RECTANGLE);
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen,
+            Operation operation) throws HyracksDataException {
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(rightBytes[rightStart]);
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNull(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (comparisonUndefined(leftTag, rightTag, operation)) {
+            return Result.NULL;
+        }
+        // compare number if one of args is number
+        comparisonResult =
+                LogicalComparatorUtil.compareNumbers(leftTag, leftBytes, leftStart, rightTag, rightBytes, rightStart);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+
+        // comparing non-numeric
+        // return null if !=, the assumption here is only numeric types are compatible with each other
+        if (leftTag != rightTag) {
+            return Result.NULL;
+        }
+
+        leftStart++;
+        leftLen--;
+        rightStart++;
+        rightLen--;
+
+        int result;
+        switch (leftTag) {
+            case BOOLEAN:
+                result = Integer.compare(leftBytes[leftStart], rightBytes[rightStart]);
+                break;
+            case STRING:
+                result = strBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case YEARMONTHDURATION:
+            case TIME:
+            case DATE:
+                result = Integer.compare(AInt32SerializerDeserializer.getInt(leftBytes, leftStart),
+                        AInt32SerializerDeserializer.getInt(rightBytes, rightStart));
+                break;
+            case DAYTIMEDURATION:
+            case DATETIME:
+                result = Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftStart),
+                        AInt64SerializerDeserializer.getLong(rightBytes, rightStart));
+                break;
+            case CIRCLE:
+                result = circleBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case LINE:
+                result = lineBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case POINT:
+                result = pointBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case POINT3D:
+                result = point3DBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart,
+                        rightLen);
+                break;
+            case POLYGON:
+                result = polygonBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart,
+                        rightLen);
+                break;
+            case DURATION:
+                result = durationBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case INTERVAL:
+                result = intervalBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case RECTANGLE:
+                result = rectangleBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart,
+                        rightLen);
+                break;
+            case BINARY:
+                result = byteArrayComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case UUID:
+                result = uuidBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            default:
+                return Result.NULL;
+        }
+        return ILogicalBinaryComparator.returnResult(result);
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant, Operation operation) {
+        // TODO(ali): currently defined for numbers only
+        if (rightConstant == null) {
+            return Result.NULL;
+        }
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNull(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (comparisonUndefined(leftTag, rightTag, operation)) {
+            return Result.NULL;
+        }
+        comparisonResult = LogicalComparatorUtil.compareNumWithConstant(leftTag, leftBytes, leftStart, rightConstant);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen, Operation operation) {
+        // TODO(ali): currently defined for numbers only
+        Result result = compare(rightBytes, rightStart, rightLen, leftConstant, operation);
+        if (result == Result.LT) {
+            return Result.GT;
+        } else if (result == Result.GT) {
+            return Result.LT;
+        }
+        return result;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, IAObject rightConstant, Operation operation) {
+        // TODO(ali): currently defined for numbers only
+        if (leftConstant == null || rightConstant == null) {
+            // illegal state maybe???
+            return Result.NULL;
+        }
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNull(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (comparisonUndefined(leftTag, rightTag, operation)) {
+            return Result.NULL;
+        }
+        comparisonResult = LogicalComparatorUtil.compareConstants(leftConstant, rightConstant);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+
+    private static boolean comparisonUndefined(ATypeTag leftTag, ATypeTag rightTag, Operation op) {
+        return op == Operation.INEQUALITY && (UNDEFINED_TYPES.contains(leftTag) || UNDEFINED_TYPES.contains(rightTag));
+    }
+}
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 50e3932..89fd094 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
@@ -20,50 +20,77 @@
 
 import java.io.DataOutput;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
+import org.apache.asterix.dataflow.data.common.LogicalComparatorUtil;
+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.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.evaluators.ConstantEvalFactory;
 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.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 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;
 
 public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
 
+    @SuppressWarnings("unchecked")
+    protected final ISerializerDeserializer<AMissing> missingSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+    @SuppressWarnings("unchecked")
+    protected final ISerializerDeserializer<ANull> nullSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
     protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     protected final DataOutput out = resultStorage.getDataOutput();
     protected final TaggedValuePointable argLeft = TaggedValuePointable.FACTORY.createPointable();
-    protected final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
-    protected final IPointable outLeft = VoidPointable.FACTORY.createPointable();
-    protected final IPointable outRight = VoidPointable.FACTORY.createPointable();
-    protected final IScalarEvaluator evalLeft;
-    protected final IScalarEvaluator evalRight;
+    private final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
+    private final IScalarEvaluator evalLeft;
+    private final IScalarEvaluator evalRight;
     protected final SourceLocation sourceLoc;
-    private final ComparisonHelper ch;
-    private Number leftValue;
-    private Number rightValue;
+    private final ILogicalBinaryComparator logicalComparator;
+    private final Operation operation;
+    private IAObject leftConstant;
+    private IAObject rightConstant;
 
-    public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
-            throws HyracksDataException {
+    public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory, IAType leftType,
+            IScalarEvaluatorFactory evalRightFactory, IAType rightType, IHyracksTaskContext ctx,
+            SourceLocation sourceLoc, Operation operation) throws HyracksDataException {
         this.evalLeft = evalLeftFactory.createScalarEvaluator(ctx);
         this.evalRight = evalRightFactory.createScalarEvaluator(ctx);
         this.sourceLoc = sourceLoc;
-        ch = new ComparisonHelper(sourceLoc);
-        leftValue = getValueOfConstantEval(evalLeftFactory);
-        rightValue = getValueOfConstantEval(evalRightFactory);
+        this.operation = operation;
+        logicalComparator = LogicalComparatorUtil.createLogicalComparator(leftType, rightType);
+        leftConstant = getValueOfConstantEval(evalLeftFactory);
+        rightConstant = getValueOfConstantEval(evalRightFactory);
     }
 
-    private Number getValueOfConstantEval(IScalarEvaluatorFactory factory) throws HyracksDataException {
+    private IAObject getValueOfConstantEval(IScalarEvaluatorFactory factory) {
         if (factory instanceof ConstantEvalFactory) {
-            return ch.getNumberValue(((ConstantEvalFactory) factory).getValue());
+            return getConstantValue(((ConstantEvalFactory) factory).getValue());
         }
         return null;
     }
@@ -73,27 +100,63 @@
         // Evaluates input args.
         evalLeft.evaluate(tuple, argLeft);
         evalRight.evaluate(tuple, argRight);
-        argLeft.getValue(outLeft);
-        argRight.getValue(outRight);
-
         evaluateImpl(result);
     }
 
     protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
 
-    // checks whether two types are comparable
-    boolean comparabilityCheck() {
-        // Checks whether two types are comparable or not
-        ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-        ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
-
-        // Are two types compatible, meaning that they can be compared? (e.g., compare between numeric types
-        return ATypeHierarchy.isCompatible(typeTag1, typeTag2);
+    Result compare() throws HyracksDataException {
+        if (leftConstant != null && rightConstant != null) {
+            // both are constants
+            return logicalComparator.compare(leftConstant, rightConstant, operation);
+        } else if (leftConstant != null) {
+            // left is constant, right isn't
+            return logicalComparator.compare(leftConstant, argRight.getByteArray(), argRight.getStartOffset(),
+                    argRight.getLength(), operation);
+        } else if (rightConstant != null) {
+            // right is constant, left isn't
+            return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
+                    rightConstant, operation);
+        } else {
+            return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
+                    argRight.getByteArray(), argRight.getStartOffset(), argRight.getLength(), operation);
+        }
     }
 
-    int compare() throws HyracksDataException {
-        ATypeTag leftTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-        ATypeTag rightTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
-        return ch.compare(leftTypeTag, rightTypeTag, outLeft, outRight, leftValue, rightValue);
+    void writeMissing(IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        missingSerde.serialize(AMissing.MISSING, out);
+        result.set(resultStorage);
+    }
+
+    void writeNull(IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        nullSerde.serialize(ANull.NULL, out);
+        result.set(resultStorage);
+    }
+
+    private IAObject getConstantValue(byte[] bytes) {
+        int start = 0;
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[start]);
+        if (typeTag == null) {
+            return null;
+        }
+        start++;
+        switch (typeTag) {
+            case TINYINT:
+                return new AInt8(AInt8SerializerDeserializer.getByte(bytes, start));
+            case SMALLINT:
+                return new AInt16(AInt16SerializerDeserializer.getShort(bytes, start));
+            case INTEGER:
+                return new AInt32(AInt32SerializerDeserializer.getInt(bytes, start));
+            case BIGINT:
+                return new AInt64(AInt64SerializerDeserializer.getLong(bytes, start));
+            case FLOAT:
+                return new AFloat(AFloatSerializerDeserializer.getFloat(bytes, start));
+            case DOUBLE:
+                return new ADouble(ADoubleSerializerDeserializer.getDouble(bytes, start));
+            default:
+                return null;
+        }
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
index 27ae471..40c5ab9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
@@ -19,6 +19,9 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
+import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -27,14 +30,20 @@
 
 public abstract class AbstractIfEqualsEvaluator extends AbstractComparisonEvaluator {
 
-    AbstractIfEqualsEvaluator(IScalarEvaluatorFactory evalLeftFactory, IScalarEvaluatorFactory evalRightFactory,
-            IHyracksTaskContext ctx, SourceLocation sourceLoc) throws HyracksDataException {
-        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    AbstractIfEqualsEvaluator(IScalarEvaluatorFactory evalLeftFactory, IAType leftType,
+            IScalarEvaluatorFactory evalRightFactory, IAType rightType, IHyracksTaskContext ctx,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(evalLeftFactory, leftType, evalRightFactory, rightType, ctx, sourceLoc, Operation.EQUALITY);
     }
 
     @Override
     protected void evaluateImpl(IPointable result) throws HyracksDataException {
-        if (comparabilityCheck() && compare() == 0) {
+        Result comparisonResult = compare();
+        if (comparisonResult == Result.MISSING) {
+            writeMissing(result);
+        } else if (comparisonResult == Result.NULL) {
+            writeNull(result);
+        } else if (comparisonResult == Result.EQ) {
             resultStorage.reset();
             writeEqualsResult();
             result.set(resultStorage);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
index 2d8c499..5b27ed8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
@@ -19,13 +19,12 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -35,64 +34,29 @@
 
 public abstract class AbstractValueComparisonEvaluator extends AbstractComparisonEvaluator {
     @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ABoolean> serde =
+    private ISerializerDeserializer<ABoolean> serde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ANull> nullSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-    public AbstractValueComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
-            throws HyracksDataException {
-        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    public AbstractValueComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory, IAType leftType,
+            IScalarEvaluatorFactory evalRightFactory, IAType rightType, IHyracksTaskContext ctx,
+            SourceLocation sourceLoc, Operation operation) throws HyracksDataException {
+        super(evalLeftFactory, leftType, evalRightFactory, rightType, ctx, sourceLoc, operation);
     }
 
     @Override
     protected void evaluateImpl(IPointable result) throws HyracksDataException {
-        resultStorage.reset();
-
-        // 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
-        if (comparabilityCheck()) {
-            // Two types can be compared
-            int r = compare();
-            ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
-            serde.serialize(b, out);
+        Result comparisonResult = compare();
+        if (comparisonResult == Result.MISSING) {
+            writeMissing(result);
+        } else if (comparisonResult == Result.NULL || comparisonResult == Result.MISMATCH) {
+            writeNull(result);
         } else {
-            // result:NULL - two types cannot be compared.
-            nullSerde.serialize(ANull.NULL, out);
-        }
-        result.set(resultStorage);
-    }
-
-    protected abstract boolean isTotallyOrderable();
-
-    protected abstract boolean getComparisonResult(int r);
-
-    // 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 HyracksDataException {
-        if (argLeft.getLength() != 0) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-            switch (typeTag) {
-                case DURATION:
-                case INTERVAL:
-                case LINE:
-                case POINT:
-                case POINT3D:
-                case POLYGON:
-                case CIRCLE:
-                case RECTANGLE:
-                    throw new UnsupportedTypeException(sourceLoc, ComparisonHelper.COMPARISON, argLeft.getTag());
-                default:
-                    return;
-            }
+            resultStorage.reset();
+            ABoolean b = getComparisonResult(comparisonResult) ? ABoolean.TRUE : ABoolean.FALSE;
+            serde.serialize(b, out);
+            result.set(resultStorage);
         }
     }
 
+    protected abstract boolean getComparisonResult(Result r);
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
index f38c3e5..e97c313 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
@@ -19,10 +19,15 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -31,10 +36,18 @@
 
 public class EqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new EqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -44,22 +57,24 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        Operation.EQUALITY) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r == 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return false;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
index 3815666..b5095f6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
@@ -19,10 +19,15 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -31,10 +36,18 @@
 
 public class GreaterThanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GreaterThanDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -44,22 +57,24 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        Operation.INEQUALITY) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r > 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.GT;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
index 1598165..7083292 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
@@ -19,10 +19,15 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -31,10 +36,18 @@
 
 public class GreaterThanOrEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GreaterThanOrEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -44,22 +57,24 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        Operation.INEQUALITY) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r >= 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.GT || r == Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
index 4198934..751aa0f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
@@ -19,10 +19,15 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -31,10 +36,18 @@
 
 public class LessThanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LessThanDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -44,22 +57,24 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        Operation.INEQUALITY) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r < 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.LT;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
index 478c652..bb5deca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
@@ -19,10 +19,15 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -31,10 +36,18 @@
 
 public class LessThanOrEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LessThanOrEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -44,22 +57,24 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        Operation.INEQUALITY) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r <= 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.LT || r == Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
index 7ed194c..13c87f4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
@@ -19,26 +19,34 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class MissingIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new MissingIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -48,17 +56,19 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
-
-                    @SuppressWarnings("unchecked")
-                    final ISerializerDeserializer<AMissing> missingSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     @Override
                     protected void writeEqualsResult() throws HyracksDataException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
index af50678..9d25858 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
@@ -25,8 +25,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -36,10 +39,18 @@
 
 public class NanIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NanIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -49,13 +60,19 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     final AMutableDouble equalsResult = new AMutableDouble(Double.NaN);
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
index a70bc7b..d77cd4c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
@@ -25,8 +25,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -36,10 +39,18 @@
 
 public class NegInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NegInfIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -49,13 +60,19 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     final AMutableDouble equalsResult = new AMutableDouble(Double.NEGATIVE_INFINITY);
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
index 6e03ef8..5216869 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
@@ -19,10 +19,15 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Operation;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -31,10 +36,18 @@
 
 public class NotEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NotEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -44,22 +57,24 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        Operation.EQUALITY) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r != 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return false;
+                    protected boolean getComparisonResult(Result r) {
+                        return r != Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
index 5df5142..d990831 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
@@ -19,26 +19,34 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class NullIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NullIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -48,17 +56,19 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
-
-                    @SuppressWarnings("unchecked")
-                    final ISerializerDeserializer<ANull> nullSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     @Override
                     protected void writeEqualsResult() throws HyracksDataException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
index a1f4f13..52c41bb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
@@ -25,8 +25,11 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -36,10 +39,18 @@
 
 public class PosInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
+    private IAType leftType;
+    private IAType rightType;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new PosInfIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -49,13 +60,19 @@
     }
 
     @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+
+    @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     final AMutableDouble equalsResult = new AMutableDouble(Double.POSITIVE_INFINITY);
 

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

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


Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/42c9aM : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4979/ (2/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3: Contrib-2

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

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/455/ (13/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5038/ (11/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5650/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5377/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5623/ (8/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


[ASTERIXDB-2516][RT] add support for deep comparison 1

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

Details:
Add support for deep comparison.
- add interface ILogicalBinaryComparator
- switch EQ,LT,GT and other functions to use ILogicalBinaryComparator
instead of ComparisonHelper

Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3145
Reviewed-by: Till Westmann <ti...@apache.org>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
Reviewed-by: Murtadha Hubail <mh...@apache.org>
Contrib: Michael Blow <mb...@apache.org>
Integration-Tests: Michael Blow <mb...@apache.org>
Tested-by: Michael Blow <mb...@apache.org>
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
33 files changed, 1,016 insertions(+), 217 deletions(-)

Approvals:
  Till Westmann: Looks good to me, but someone else must approve
  Michael Blow: Verified; ; Verified
  Murtadha Hubail: Looks good to me, approved
  Dmitry Lychagin: Looks good to me, but someone else must approve



diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
index f302cef..5d40207 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
@@ -22,14 +22,18 @@
 
 import static org.mockito.Mockito.mock;
 
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 
+import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionCollection;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.evaluators.ConstantEvalFactory;
@@ -46,13 +50,14 @@
         List<IFunctionDescriptorFactory> functions =
                 FunctionCollection.createDefaultFunctionCollection().getFunctionDescriptorFactories();
         int testedFunctions = 0;
+        Set<FunctionIdentifier> excluded = new HashSet<>();
+        buildExcluded(excluded);
         for (IFunctionDescriptorFactory func : functions) {
             String className = func.getClass().getName();
             // We test all generated functions except
             // record and cast functions, which requires type settings (we test them in runtime tests).
             if (className.contains("Gen") && !className.contains("record") && !className.contains("Cast")) {
-                System.out.println("Testing " + className);
-                testFunction(func);
+                testFunction(func, excluded, className);
                 ++testedFunctions;
             }
         }
@@ -61,11 +66,15 @@
                 testedFunctions >= 217);
     }
 
-    private void testFunction(IFunctionDescriptorFactory funcFactory) throws Exception {
+    private void testFunction(IFunctionDescriptorFactory funcFactory, Set<FunctionIdentifier> excluded,
+            String className) throws Exception {
         IFunctionDescriptor functionDescriptor = funcFactory.createFunctionDescriptor();
-        if (!(functionDescriptor instanceof AbstractScalarFunctionDynamicDescriptor)) {
+        if (!(functionDescriptor instanceof AbstractScalarFunctionDynamicDescriptor)
+                || excluded.contains(functionDescriptor.getIdentifier())) {
+            System.out.println("Excluding " + className);
             return;
         }
+        System.out.println("Testing " + className);
         AbstractScalarFunctionDynamicDescriptor funcDesc = (AbstractScalarFunctionDynamicDescriptor) functionDescriptor;
         int inputArity = funcDesc.getIdentifier().getArity();
         Iterator<IScalarEvaluatorFactory[]> argEvalFactoryIterator = getArgCombinations(inputArity);
@@ -114,4 +123,18 @@
 
     }
 
+    // adds functions that require setImmutables be called in order to set the args types
+    private void buildExcluded(Set<FunctionIdentifier> excluded) {
+        excluded.add(BuiltinFunctions.EQ);
+        excluded.add(BuiltinFunctions.LT);
+        excluded.add(BuiltinFunctions.GT);
+        excluded.add(BuiltinFunctions.GE);
+        excluded.add(BuiltinFunctions.LE);
+        excluded.add(BuiltinFunctions.NEQ);
+        excluded.add(BuiltinFunctions.MISSING_IF);
+        excluded.add(BuiltinFunctions.NAN_IF);
+        excluded.add(BuiltinFunctions.NEGINF_IF);
+        excluded.add(BuiltinFunctions.NULL_IF);
+        excluded.add(BuiltinFunctions.POSINF_IF);
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
index 58fd49d..56df4ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
@@ -177,43 +177,36 @@
   <test-case FilePath="comparison">
     <compilation-unit name="issue363_inequality_duration">
       <output-dir compare="Text">issue363_inequality_duration</output-dir>
-      <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type duration</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="comparison">
     <compilation-unit name="issue363_inequality_interval">
       <output-dir compare="Text">issue363_inequality_interval</output-dir>
-      <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type interval</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="comparison">
     <compilation-unit name="issue363_inequality_point">
       <output-dir compare="Text">issue363_inequality_point</output-dir>
-      <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type point</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="comparison">
     <compilation-unit name="issue363_inequality_line">
       <output-dir compare="Text">issue363_inequality_line</output-dir>
-      <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type line</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="comparison">
     <compilation-unit name="issue363_inequality_polygon">
       <output-dir compare="Text">issue363_inequality_polygon</output-dir>
-      <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type polygon</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="comparison">
     <compilation-unit name="issue363_inequality_rectangle">
       <output-dir compare="Text">issue363_inequality_rectangle</output-dir>
-      <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type rectangle</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="comparison">
     <compilation-unit name="issue363_inequality_circle">
       <output-dir compare="Text">issue363_inequality_circle</output-dir>
-      <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type circle</expected-error>
     </compilation-unit>
   </test-case>
   <test-case FilePath="comparison">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
index e3b97f5..e3846b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
@@ -1 +1 @@
-[  ]
+{ "circle0": null, "circle1": null, "circle2": null, "circle3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
index e3b97f5..60e80d9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
@@ -1 +1 @@
-[  ]
+{ "duration0": null, "duration1": null, "duration2": null, "duration3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
index e3b97f5..76c2bb0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
@@ -1 +1 @@
-[  ]
+{ "interval0": null, "interval1": null, "interval2": null, "interval3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
index e3b97f5..31cf931 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
@@ -1 +1 @@
-[  ]
+{ "line0": null, "line1": null, "line2": null, "line3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
index e3b97f5..3e24175 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
@@ -1 +1 @@
-[  ]
+{ "point0": null, "point1": null, "point2": null, "point3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
index e3b97f5..8a0655b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
@@ -1 +1 @@
-[  ]
+{ "polygon0": null, "polygon1": null, "polygon2": null, "polygon3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
index e3b97f5..5429dcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
@@ -1 +1 @@
-[  ]
+{ "rectangle0": null, "rectangle1": null, "rectangle2": null, "rectangle3": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index f753bd0..24c2f28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -2303,43 +2303,36 @@
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_duration">
         <output-dir compare="Text">issue363_inequality_duration</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type duration</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_interval">
         <output-dir compare="Text">issue363_inequality_interval</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type interval</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_point">
         <output-dir compare="Text">issue363_inequality_point</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type point</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_line">
         <output-dir compare="Text">issue363_inequality_line</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type line</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_polygon">
         <output-dir compare="Text">issue363_inequality_polygon</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type polygon</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_rectangle">
         <output-dir compare="Text">issue363_inequality_rectangle</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type rectangle</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_circle">
         <output-dir compare="Text">issue363_inequality_circle</output-dir>
-        <expected-error>Unsupported type: comparison operations (&gt;, &gt;=, &lt;, and &lt;=) cannot process input type circle</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
index aef4a79..8f721cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
@@ -1672,43 +1672,36 @@
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_duration">
         <output-dir compare="AST">issue363_inequality_duration</output-dir>
-        <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_interval">
         <output-dir compare="AST">issue363_inequality_interval</output-dir>
-        <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_point">
         <output-dir compare="AST">issue363_inequality_point</output-dir>
-        <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_line">
         <output-dir compare="AST">issue363_inequality_line</output-dir>
-        <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_polygon">
         <output-dir compare="AST">issue363_inequality_polygon</output-dir>
-        <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_rectangle">
         <output-dir compare="AST">issue363_inequality_rectangle</output-dir>
-        <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
       <compilation-unit name="issue363_inequality_circle">
         <output-dir compare="AST">issue363_inequality_circle</output-dir>
-        <expected-error>org.apache.asterix.common.exceptions.AsterixException</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="comparison">
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
new file mode 100644
index 0000000..270c9dd
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
@@ -0,0 +1,46 @@
+/*
+ * 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.dataflow.data.common;
+
+import org.apache.asterix.om.base.IAObject;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILogicalBinaryComparator {
+    enum Result {
+        MISSING,
+        NULL,
+        EQ,
+        LT,
+        GT,
+        MISMATCH
+    }
+
+    static Result asResult(int result) {
+        return result < 0 ? Result.LT : (result == 0 ? Result.EQ : Result.GT);
+    }
+
+    Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen)
+            throws HyracksDataException;
+
+    Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant);
+
+    Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen);
+
+    Result compare(IAObject leftConstant, IAObject rightConstant);
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
new file mode 100644
index 0000000..ddbabb2
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
@@ -0,0 +1,218 @@
+/*
+ * 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.dataflow.data.nontagged.comparators;
+
+import static org.apache.asterix.om.types.ATypeTag.BIGINT;
+import static org.apache.asterix.om.types.ATypeTag.DOUBLE;
+import static org.apache.asterix.om.types.ATypeTag.FLOAT;
+import static org.apache.asterix.om.types.ATypeTag.INTEGER;
+import static org.apache.asterix.om.types.ATypeTag.MISSING;
+import static org.apache.asterix.om.types.ATypeTag.NULL;
+import static org.apache.asterix.om.types.ATypeTag.SMALLINT;
+import static org.apache.asterix.om.types.ATypeTag.TINYINT;
+
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+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.om.base.ADouble;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+
+public class LogicalComparatorUtil {
+
+    private LogicalComparatorUtil() {
+    }
+
+    public static ILogicalBinaryComparator createLogicalComparator(IAType left, IAType right, boolean isEquality) {
+        // this should never happen, but for back-compat where types are missing and would be defaulted to null
+        if (left == null || right == null) {
+            return new LogicalScalarBinaryComparator(isEquality);
+        }
+        IAType leftType = TypeComputeUtils.getActualType(left);
+        IAType rightType = TypeComputeUtils.getActualType(right);
+
+        // TODO(ali): after making comparators in scalar comparator stateless, create an INSTANCE only and use it here
+        if (leftType.getTypeTag().isDerivedType() && rightType.getTypeTag().isDerivedType()) {
+            return new LogicalComplexBinaryComparator(leftType, rightType, isEquality);
+        } else if (leftType.getTypeTag() == ATypeTag.ANY || rightType.getTypeTag() == ATypeTag.ANY) {
+            return new LogicalGenericBinaryComparator(leftType, rightType, isEquality);
+        } else {
+            return new LogicalScalarBinaryComparator(isEquality);
+        }
+    }
+
+    static ILogicalBinaryComparator.Result returnMissingOrNullOrMismatch(ATypeTag leftTag, ATypeTag rightTag) {
+        if (leftTag == MISSING || rightTag == MISSING) {
+            return ILogicalBinaryComparator.Result.MISSING;
+        }
+        if (leftTag == NULL || rightTag == NULL) {
+            return ILogicalBinaryComparator.Result.NULL;
+        }
+        if (!ATypeHierarchy.isCompatible(leftTag, rightTag)) {
+            return ILogicalBinaryComparator.Result.MISMATCH;
+        }
+        return null;
+    }
+
+    // checking that left and right are compatible has to be done before calling this
+    static ILogicalBinaryComparator.Result compareNumbers(ATypeTag leftTag, byte[] b1, int s1, ATypeTag rightTag,
+            byte[] b2, int s2) {
+        int result;
+        if (leftTag == DOUBLE || rightTag == DOUBLE) {
+            result = Double.compare(getDoubleValue(leftTag, b1, s1), getDoubleValue(rightTag, b2, s2));
+        } else if (leftTag == FLOAT || rightTag == FLOAT) {
+            result = Float.compare((float) getDoubleValue(leftTag, b1, s1), (float) getDoubleValue(rightTag, b2, s2));
+        } else if (leftTag == BIGINT || rightTag == BIGINT) {
+            result = Long.compare(getLongValue(leftTag, b1, s1), getLongValue(rightTag, b2, s2));
+        } else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
+            result = Integer.compare((int) getLongValue(leftTag, b1, s1), (int) getLongValue(rightTag, b2, s2));
+        } else {
+            return null;
+        }
+        return ILogicalBinaryComparator.asResult(result);
+    }
+
+    // checking that left and right are compatible has to be done before calling this
+    static ILogicalBinaryComparator.Result compareNumWithConstant(ATypeTag leftTag, byte[] b1, int s1,
+            IAObject rightConstant) {
+        int result;
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag == DOUBLE || rightTag == DOUBLE) {
+            result = Double.compare(getDoubleValue(leftTag, b1, s1), getConstantDouble(rightConstant));
+        } else if (leftTag == FLOAT || rightTag == FLOAT) {
+            result = Float.compare((float) getDoubleValue(leftTag, b1, s1), (float) getConstantDouble(rightConstant));
+        } else if (leftTag == BIGINT || rightTag == BIGINT) {
+            result = Long.compare(getLongValue(leftTag, b1, s1), getConstantLong(rightConstant));
+        } else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
+            result = Integer.compare((int) getLongValue(leftTag, b1, s1), (int) getConstantLong(rightConstant));
+        } else {
+            return null;
+        }
+        return ILogicalBinaryComparator.asResult(result);
+    }
+
+    // checking that left and right are compatible has to be done before calling this
+    static ILogicalBinaryComparator.Result compareConstants(IAObject leftConstant, IAObject rightConstant) {
+        int result;
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag == DOUBLE || rightTag == DOUBLE) {
+            result = Double.compare(getConstantDouble(leftConstant), getConstantDouble(rightConstant));
+        } else if (leftTag == FLOAT || rightTag == FLOAT) {
+            result = Float.compare((float) getConstantDouble(leftConstant), (float) getConstantDouble(rightConstant));
+        } else if (leftTag == BIGINT || rightTag == BIGINT) {
+            result = Long.compare(getConstantLong(leftConstant), getConstantLong(rightConstant));
+        } else if (leftTag == INTEGER || leftTag == SMALLINT || leftTag == TINYINT) {
+            result = Integer.compare((int) getConstantLong(leftConstant), (int) getConstantLong(rightConstant));
+        } else {
+            return null;
+        }
+        return ILogicalBinaryComparator.asResult(result);
+    }
+
+    @SuppressWarnings("squid:S1226") // asking for introducing a new variable for s
+    private static double getDoubleValue(ATypeTag numericTag, byte[] b, int s) {
+        s++;
+        switch (numericTag) {
+            case TINYINT:
+                return AInt8SerializerDeserializer.getByte(b, s);
+            case SMALLINT:
+                return AInt16SerializerDeserializer.getShort(b, s);
+            case INTEGER:
+                return AInt32SerializerDeserializer.getInt(b, s);
+            case BIGINT:
+                return AInt64SerializerDeserializer.getLong(b, s);
+            case FLOAT:
+                return AFloatSerializerDeserializer.getFloat(b, s);
+            case DOUBLE:
+                return ADoubleSerializerDeserializer.getDouble(b, s);
+            default:
+                // TODO(ali): use unsupported type
+                throw new UnsupportedOperationException();
+        }
+    }
+
+    @SuppressWarnings("squid:S1226") // asking for introducing a new variable for s
+    private static long getLongValue(ATypeTag numericTag, byte[] b, int s) {
+        s++;
+        switch (numericTag) {
+            case TINYINT:
+                return AInt8SerializerDeserializer.getByte(b, s);
+            case SMALLINT:
+                return AInt16SerializerDeserializer.getShort(b, s);
+            case INTEGER:
+                return AInt32SerializerDeserializer.getInt(b, s);
+            case BIGINT:
+                return AInt64SerializerDeserializer.getLong(b, s);
+            default:
+                // TODO(ali): use unsupported type
+                throw new UnsupportedOperationException();
+        }
+    }
+
+    private static double getConstantDouble(IAObject numeric) {
+        ATypeTag tag = numeric.getType().getTypeTag();
+        switch (tag) {
+            case DOUBLE:
+                return ((ADouble) numeric).getDoubleValue();
+            case FLOAT:
+                return ((AFloat) numeric).getFloatValue();
+            case BIGINT:
+                return ((AInt64) numeric).getLongValue();
+            case INTEGER:
+                return ((AInt32) numeric).getIntegerValue();
+            case SMALLINT:
+                return ((AInt16) numeric).getShortValue();
+            case TINYINT:
+                return ((AInt8) numeric).getByteValue();
+            default:
+                // TODO(ali): use unsupported type
+                throw new UnsupportedOperationException();
+        }
+    }
+
+    private static long getConstantLong(IAObject numeric) {
+        ATypeTag tag = numeric.getType().getTypeTag();
+        switch (tag) {
+            case BIGINT:
+                return ((AInt64) numeric).getLongValue();
+            case INTEGER:
+                return ((AInt32) numeric).getIntegerValue();
+            case SMALLINT:
+                return ((AInt16) numeric).getShortValue();
+            case TINYINT:
+                return ((AInt8) numeric).getByteValue();
+            default:
+                // TODO(ali): use unsupported type
+                throw new UnsupportedOperationException();
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
new file mode 100644
index 0000000..050685f
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
@@ -0,0 +1,97 @@
+/*
+ * 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.dataflow.data.nontagged.comparators;
+
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LogicalComplexBinaryComparator implements ILogicalBinaryComparator {
+
+    @SuppressWarnings("squid:S1068") // unused variable, remove once used
+    private final IAType leftType;
+    @SuppressWarnings("squid:S1068") // unused variable, remove once used
+    private final IAType rightType;
+    @SuppressWarnings("squid:S1068") // unused variable, remove once used
+    private final boolean isEquality;
+    @SuppressWarnings("squid:S1068") // unused variable, remove once used
+    private final LogicalScalarBinaryComparator scalarComparator;
+
+    public LogicalComplexBinaryComparator(IAType leftType, IAType rightType, boolean isEquality) {
+        this.leftType = leftType;
+        this.rightType = rightType;
+        this.isEquality = isEquality;
+        this.scalarComparator = new LogicalScalarBinaryComparator(isEquality);
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen)
+            throws HyracksDataException {
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(rightBytes[rightStart]);
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNullOrMismatch(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (!leftTag.isDerivedType() || !rightTag.isDerivedType()) {
+            return Result.NULL;
+        }
+        // TODO(ali): complex types(records, arrays, multisets) logic here
+        return Result.NULL;
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant) {
+        // TODO(ali): not defined currently for constant complex types
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNullOrMismatch(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen) {
+        // TODO(ali): not defined currently for constant complex types
+        Result result = compare(rightBytes, rightStart, rightLen, leftConstant);
+        if (result == Result.LT) {
+            return Result.GT;
+        } else if (result == Result.GT) {
+            return Result.LT;
+        }
+        return result;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, IAObject rightConstant) {
+        // TODO(ali): not defined currently for constant complex types
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNullOrMismatch(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
new file mode 100644
index 0000000..99c4151
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
@@ -0,0 +1,79 @@
+/*
+ * 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.dataflow.data.nontagged.comparators;
+
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LogicalGenericBinaryComparator implements ILogicalBinaryComparator {
+
+    private final LogicalComplexBinaryComparator complexComparator;
+    private final LogicalScalarBinaryComparator scalarComparator;
+
+    public LogicalGenericBinaryComparator(IAType leftType, IAType rightType, boolean isEquality) {
+        complexComparator = new LogicalComplexBinaryComparator(leftType, rightType, isEquality);
+        scalarComparator = new LogicalScalarBinaryComparator(isEquality);
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen)
+            throws HyracksDataException {
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(rightBytes[rightStart]);
+        if (leftTag.isDerivedType() && rightTag.isDerivedType()) {
+            return complexComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+        }
+        return scalarComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant) {
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag.isDerivedType() && rightTag.isDerivedType()) {
+            return complexComparator.compare(leftBytes, leftStart, leftLen, rightConstant);
+        }
+        return scalarComparator.compare(leftBytes, leftStart, leftLen, rightConstant);
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen) {
+        Result result = compare(rightBytes, rightStart, rightLen, leftConstant);
+        if (result == Result.LT) {
+            return Result.GT;
+        } else if (result == Result.GT) {
+            return Result.LT;
+        }
+        return result;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, IAObject rightConstant) {
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        if (leftTag.isDerivedType() && rightTag.isDerivedType()) {
+            return complexComparator.compare(leftConstant, rightConstant);
+        }
+        return scalarComparator.compare(leftConstant, rightConstant);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
new file mode 100644
index 0000000..b6f2634
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
@@ -0,0 +1,233 @@
+/*
+ * 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.dataflow.data.nontagged.comparators;
+
+import static org.apache.asterix.om.types.ATypeTag.CIRCLE;
+import static org.apache.asterix.om.types.ATypeTag.DURATION;
+import static org.apache.asterix.om.types.ATypeTag.INTERVAL;
+import static org.apache.asterix.om.types.ATypeTag.LINE;
+import static org.apache.asterix.om.types.ATypeTag.POINT;
+import static org.apache.asterix.om.types.ATypeTag.POINT3D;
+import static org.apache.asterix.om.types.ATypeTag.POLYGON;
+import static org.apache.asterix.om.types.ATypeTag.RECTANGLE;
+
+import java.util.EnumSet;
+
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AYearMonthDurationSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+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.primitive.ByteArrayPointable;
+
+public class LogicalScalarBinaryComparator implements ILogicalBinaryComparator {
+
+    private static final EnumSet<ATypeTag> INEQUALITY_UNDEFINED_TYPES =
+            EnumSet.of(DURATION, INTERVAL, LINE, POINT, POINT3D, POLYGON, CIRCLE, RECTANGLE);
+
+    private final IBinaryComparator strBinaryComp =
+            BinaryComparatorFactoryProvider.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();
+
+    private final boolean isEquality;
+
+    public LogicalScalarBinaryComparator(boolean isEquality) {
+        this.isEquality = isEquality;
+    }
+
+    @SuppressWarnings("squid:S1226") // asking for introducing a new variable for incremented local variables
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen)
+            throws HyracksDataException {
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(rightBytes[rightStart]);
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNullOrMismatch(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (comparisonUndefined(leftTag, rightTag, isEquality)) {
+            return Result.NULL;
+        }
+        // compare number if one of args is number
+        comparisonResult =
+                LogicalComparatorUtil.compareNumbers(leftTag, leftBytes, leftStart, rightTag, rightBytes, rightStart);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+
+        // comparing non-numeric
+        // return null if !=, the assumption here is only numeric types are compatible with each other
+        if (leftTag != rightTag) {
+            return Result.NULL;
+        }
+
+        leftStart++;
+        leftLen--;
+        rightStart++;
+        rightLen--;
+
+        int result;
+        switch (leftTag) {
+            case BOOLEAN:
+                result = Integer.compare(leftBytes[leftStart], rightBytes[rightStart]);
+                break;
+            case STRING:
+                result = strBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case YEARMONTHDURATION:
+                result = Integer.compare(AYearMonthDurationSerializerDeserializer.getYearMonth(leftBytes, leftStart),
+                        AYearMonthDurationSerializerDeserializer.getYearMonth(rightBytes, rightStart));
+                break;
+            case TIME:
+                result = Integer.compare(ATimeSerializerDeserializer.getChronon(leftBytes, leftStart),
+                        ATimeSerializerDeserializer.getChronon(rightBytes, rightStart));
+                break;
+            case DATE:
+                result = Integer.compare(ADateSerializerDeserializer.getChronon(leftBytes, leftStart),
+                        ADateSerializerDeserializer.getChronon(rightBytes, rightStart));
+                break;
+            case DAYTIMEDURATION:
+                result = Long.compare(ADayTimeDurationSerializerDeserializer.getDayTime(leftBytes, leftStart),
+                        ADayTimeDurationSerializerDeserializer.getDayTime(rightBytes, rightStart));
+                break;
+            case DATETIME:
+                result = Long.compare(ADateTimeSerializerDeserializer.getChronon(leftBytes, leftStart),
+                        ADateTimeSerializerDeserializer.getChronon(rightBytes, rightStart));
+                break;
+            case CIRCLE:
+                result = circleBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case LINE:
+                result = lineBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case POINT:
+                result = pointBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case POINT3D:
+                result = point3DBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart,
+                        rightLen);
+                break;
+            case POLYGON:
+                result = polygonBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart,
+                        rightLen);
+                break;
+            case DURATION:
+                result = durationBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case INTERVAL:
+                result = intervalBinaryComp.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case RECTANGLE:
+                result = rectangleBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart,
+                        rightLen);
+                break;
+            case BINARY:
+                result = byteArrayComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            case UUID:
+                result = uuidBinaryComparator.compare(leftBytes, leftStart, leftLen, rightBytes, rightStart, rightLen);
+                break;
+            default:
+                return Result.NULL;
+        }
+        return ILogicalBinaryComparator.asResult(result);
+    }
+
+    @Override
+    public Result compare(byte[] leftBytes, int leftStart, int leftLen, IAObject rightConstant) {
+        // TODO(ali): currently defined for numbers only
+        ATypeTag leftTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(leftBytes[leftStart]);
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNullOrMismatch(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (comparisonUndefined(leftTag, rightTag, isEquality)) {
+            return Result.NULL;
+        }
+        comparisonResult = LogicalComparatorUtil.compareNumWithConstant(leftTag, leftBytes, leftStart, rightConstant);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen) {
+        // TODO(ali): currently defined for numbers only
+        Result result = compare(rightBytes, rightStart, rightLen, leftConstant);
+        if (result == Result.LT) {
+            return Result.GT;
+        } else if (result == Result.GT) {
+            return Result.LT;
+        }
+        return result;
+    }
+
+    @Override
+    public Result compare(IAObject leftConstant, IAObject rightConstant) {
+        // TODO(ali): currently defined for numbers only
+        ATypeTag leftTag = leftConstant.getType().getTypeTag();
+        ATypeTag rightTag = rightConstant.getType().getTypeTag();
+        Result comparisonResult = LogicalComparatorUtil.returnMissingOrNullOrMismatch(leftTag, rightTag);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        if (comparisonUndefined(leftTag, rightTag, isEquality)) {
+            return Result.NULL;
+        }
+        comparisonResult = LogicalComparatorUtil.compareConstants(leftConstant, rightConstant);
+        if (comparisonResult != null) {
+            return comparisonResult;
+        }
+        return Result.NULL;
+    }
+
+    private static boolean comparisonUndefined(ATypeTag leftTag, ATypeTag rightTag, boolean isEquality) {
+        return !isEquality
+                && (INEQUALITY_UNDEFINED_TYPES.contains(leftTag) || INEQUALITY_UNDEFINED_TYPES.contains(rightTag));
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java
new file mode 100644
index 0000000..a0aeb93
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.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.asterix.runtime.evaluators.comparisons;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+
+public abstract class AbstractComparisonDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    protected IAType leftType;
+    protected IAType rightType;
+
+    @Override
+    public void setImmutableStates(Object... types) {
+        leftType = (IAType) types[0];
+        rightType = (IAType) types[1];
+    }
+}
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 50e3932..b7ed05d 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
@@ -20,50 +20,74 @@
 
 import java.io.DataOutput;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
+import org.apache.asterix.dataflow.data.nontagged.comparators.LogicalComparatorUtil;
+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.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AMissing;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.evaluators.ConstantEvalFactory;
 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.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
 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;
 
 public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
 
+    @SuppressWarnings("unchecked")
+    protected final ISerializerDeserializer<AMissing> missingSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+    @SuppressWarnings("unchecked")
+    protected final ISerializerDeserializer<ANull> nullSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
     protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
     protected final DataOutput out = resultStorage.getDataOutput();
     protected final TaggedValuePointable argLeft = TaggedValuePointable.FACTORY.createPointable();
-    protected final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
-    protected final IPointable outLeft = VoidPointable.FACTORY.createPointable();
-    protected final IPointable outRight = VoidPointable.FACTORY.createPointable();
-    protected final IScalarEvaluator evalLeft;
-    protected final IScalarEvaluator evalRight;
+    private final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
+    private final IScalarEvaluator evalLeft;
+    private final IScalarEvaluator evalRight;
     protected final SourceLocation sourceLoc;
-    private final ComparisonHelper ch;
-    private Number leftValue;
-    private Number rightValue;
+    private final ILogicalBinaryComparator logicalComparator;
+    private IAObject leftConstant;
+    private IAObject rightConstant;
 
-    public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
-            throws HyracksDataException {
+    public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory, IAType leftType,
+            IScalarEvaluatorFactory evalRightFactory, IAType rightType, IHyracksTaskContext ctx,
+            SourceLocation sourceLoc, boolean isEquality) throws HyracksDataException {
         this.evalLeft = evalLeftFactory.createScalarEvaluator(ctx);
         this.evalRight = evalRightFactory.createScalarEvaluator(ctx);
         this.sourceLoc = sourceLoc;
-        ch = new ComparisonHelper(sourceLoc);
-        leftValue = getValueOfConstantEval(evalLeftFactory);
-        rightValue = getValueOfConstantEval(evalRightFactory);
+        logicalComparator = LogicalComparatorUtil.createLogicalComparator(leftType, rightType, isEquality);
+        leftConstant = getValueOfConstantEval(evalLeftFactory);
+        rightConstant = getValueOfConstantEval(evalRightFactory);
     }
 
-    private Number getValueOfConstantEval(IScalarEvaluatorFactory factory) throws HyracksDataException {
+    private IAObject getValueOfConstantEval(IScalarEvaluatorFactory factory) {
         if (factory instanceof ConstantEvalFactory) {
-            return ch.getNumberValue(((ConstantEvalFactory) factory).getValue());
+            return getConstantValue(((ConstantEvalFactory) factory).getValue());
         }
         return null;
     }
@@ -73,27 +97,67 @@
         // Evaluates input args.
         evalLeft.evaluate(tuple, argLeft);
         evalRight.evaluate(tuple, argRight);
-        argLeft.getValue(outLeft);
-        argRight.getValue(outRight);
-
         evaluateImpl(result);
     }
 
     protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
 
-    // checks whether two types are comparable
-    boolean comparabilityCheck() {
-        // Checks whether two types are comparable or not
-        ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-        ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
-
-        // Are two types compatible, meaning that they can be compared? (e.g., compare between numeric types
-        return ATypeHierarchy.isCompatible(typeTag1, typeTag2);
+    Result compare() throws HyracksDataException {
+        if (leftConstant != null) {
+            if (rightConstant != null) {
+                // both are constants
+                return logicalComparator.compare(leftConstant, rightConstant);
+            } else {
+                // left is constant, right isn't
+                return logicalComparator.compare(leftConstant, argRight.getByteArray(), argRight.getStartOffset(),
+                        argRight.getLength());
+            }
+        } else {
+            if (rightConstant != null) {
+                // right is constant, left isn't
+                return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
+                        rightConstant);
+            } else {
+                return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
+                        argRight.getByteArray(), argRight.getStartOffset(), argRight.getLength());
+            }
+        }
     }
 
-    int compare() throws HyracksDataException {
-        ATypeTag leftTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-        ATypeTag rightTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
-        return ch.compare(leftTypeTag, rightTypeTag, outLeft, outRight, leftValue, rightValue);
+    void writeMissing(IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        missingSerde.serialize(AMissing.MISSING, out);
+        result.set(resultStorage);
+    }
+
+    void writeNull(IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        nullSerde.serialize(ANull.NULL, out);
+        result.set(resultStorage);
+    }
+
+    private IAObject getConstantValue(byte[] bytes) {
+        int start = 0;
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[start]);
+        if (typeTag == null) {
+            return null;
+        }
+        start++;
+        switch (typeTag) {
+            case TINYINT:
+                return new AInt8(AInt8SerializerDeserializer.getByte(bytes, start));
+            case SMALLINT:
+                return new AInt16(AInt16SerializerDeserializer.getShort(bytes, start));
+            case INTEGER:
+                return new AInt32(AInt32SerializerDeserializer.getInt(bytes, start));
+            case BIGINT:
+                return new AInt64(AInt64SerializerDeserializer.getLong(bytes, start));
+            case FLOAT:
+                return new AFloat(AFloatSerializerDeserializer.getFloat(bytes, start));
+            case DOUBLE:
+                return new ADouble(ADoubleSerializerDeserializer.getDouble(bytes, start));
+            default:
+                return null;
+        }
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
index 27ae471..b085236 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
@@ -19,6 +19,7 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -27,19 +28,28 @@
 
 public abstract class AbstractIfEqualsEvaluator extends AbstractComparisonEvaluator {
 
-    AbstractIfEqualsEvaluator(IScalarEvaluatorFactory evalLeftFactory, IScalarEvaluatorFactory evalRightFactory,
-            IHyracksTaskContext ctx, SourceLocation sourceLoc) throws HyracksDataException {
-        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    AbstractIfEqualsEvaluator(IScalarEvaluatorFactory evalLeftFactory, IAType leftType,
+            IScalarEvaluatorFactory evalRightFactory, IAType rightType, IHyracksTaskContext ctx,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(evalLeftFactory, leftType, evalRightFactory, rightType, ctx, sourceLoc, true);
     }
 
     @Override
     protected void evaluateImpl(IPointable result) throws HyracksDataException {
-        if (comparabilityCheck() && compare() == 0) {
-            resultStorage.reset();
-            writeEqualsResult();
-            result.set(resultStorage);
-        } else {
-            result.set(argLeft);
+        switch (compare()) {
+            case MISSING:
+                writeMissing(result);
+                break;
+            case NULL:
+                writeNull(result);
+                break;
+            case EQ:
+                resultStorage.reset();
+                writeEqualsResult();
+                result.set(resultStorage);
+                break;
+            default:
+                result.set(argLeft);
         }
     }
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
index 2d8c499..dc9cdee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
@@ -19,13 +19,11 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -35,64 +33,33 @@
 
 public abstract class AbstractValueComparisonEvaluator extends AbstractComparisonEvaluator {
     @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ABoolean> serde =
+    private ISerializerDeserializer<ABoolean> serde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
-    @SuppressWarnings("unchecked")
-    protected ISerializerDeserializer<ANull> nullSerde =
-            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
-    public AbstractValueComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
-            throws HyracksDataException {
-        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+    public AbstractValueComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory, IAType leftType,
+            IScalarEvaluatorFactory evalRightFactory, IAType rightType, IHyracksTaskContext ctx,
+            SourceLocation sourceLoc, boolean isEquality) throws HyracksDataException {
+        super(evalLeftFactory, leftType, evalRightFactory, rightType, ctx, sourceLoc, isEquality);
     }
 
     @Override
     protected void evaluateImpl(IPointable result) throws HyracksDataException {
-        resultStorage.reset();
-
-        // 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
-        if (comparabilityCheck()) {
-            // Two types can be compared
-            int r = compare();
-            ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
-            serde.serialize(b, out);
-        } else {
-            // result:NULL - two types cannot be compared.
-            nullSerde.serialize(ANull.NULL, out);
-        }
-        result.set(resultStorage);
-    }
-
-    protected abstract boolean isTotallyOrderable();
-
-    protected abstract boolean getComparisonResult(int r);
-
-    // 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 HyracksDataException {
-        if (argLeft.getLength() != 0) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
-            switch (typeTag) {
-                case DURATION:
-                case INTERVAL:
-                case LINE:
-                case POINT:
-                case POINT3D:
-                case POLYGON:
-                case CIRCLE:
-                case RECTANGLE:
-                    throw new UnsupportedTypeException(sourceLoc, ComparisonHelper.COMPARISON, argLeft.getTag());
-                default:
-                    return;
-            }
+        Result comparisonResult = compare();
+        switch (comparisonResult) {
+            case MISSING:
+                writeMissing(result);
+                break;
+            case NULL:
+            case MISMATCH:
+                writeNull(result);
+                break;
+            default:
+                resultStorage.reset();
+                ABoolean b = getComparisonResult(comparisonResult) ? ABoolean.TRUE : ABoolean.FALSE;
+                serde.serialize(b, out);
+                result.set(resultStorage);
         }
     }
 
+    protected abstract boolean getComparisonResult(Result r);
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
index f38c3e5..2d33504 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
@@ -19,22 +19,30 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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;
 
-public class EqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class EqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new EqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -50,16 +58,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        true) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r == 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return false;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
index 3815666..711fb4a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
@@ -19,22 +19,30 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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;
 
-public class GreaterThanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class GreaterThanDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GreaterThanDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -50,16 +58,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        false) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r > 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.GT;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
index 1598165..930f9ef 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
@@ -19,22 +19,30 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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;
 
-public class GreaterThanOrEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class GreaterThanOrEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new GreaterThanOrEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -50,16 +58,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        false) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r >= 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.GT || r == Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
index 4198934..66a308d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
@@ -19,22 +19,30 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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;
 
-public class LessThanDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class LessThanDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LessThanDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -50,16 +58,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        false) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r < 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.LT;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
index 478c652..07147f0 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
@@ -19,22 +19,30 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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;
 
-public class LessThanOrEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class LessThanOrEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LessThanOrEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -50,16 +58,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        false) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r <= 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
+                    protected boolean getComparisonResult(Result r) {
+                        return r == Result.LT || r == Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
index 7ed194c..c339762 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
@@ -19,26 +19,30 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class MissingIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class MissingIfEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new MissingIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -54,11 +58,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
-
-                    @SuppressWarnings("unchecked")
-                    final ISerializerDeserializer<AMissing> missingSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     @Override
                     protected void writeEqualsResult() throws HyracksDataException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
index af50678..a427759 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
@@ -25,8 +25,9 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -34,12 +35,18 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class NanIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class NanIfEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NanIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -55,7 +62,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     final AMutableDouble equalsResult = new AMutableDouble(Double.NaN);
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
index a70bc7b..b66a0a1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
@@ -25,8 +25,9 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -34,12 +35,18 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class NegInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class NegInfIfEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NegInfIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -55,7 +62,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     final AMutableDouble equalsResult = new AMutableDouble(Double.NEGATIVE_INFINITY);
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
index 6e03ef8..d176298 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
@@ -19,22 +19,30 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator.Result;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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;
 
-public class NotEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class NotEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NotEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -50,16 +58,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
+                        true) {
 
                     @Override
-                    protected boolean getComparisonResult(int r) {
-                        return r != 0;
-                    }
-
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return false;
+                    protected boolean getComparisonResult(Result r) {
+                        return r != Result.EQ;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
index 5df5142..d2239ec 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
@@ -19,26 +19,31 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.ANull;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 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.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class NullIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class NullIfEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new NullIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -54,11 +59,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
-
-                    @SuppressWarnings("unchecked")
-                    final ISerializerDeserializer<ANull> nullSerde =
-                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     @Override
                     protected void writeEqualsResult() throws HyracksDataException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
index a1f4f13..6c65e9a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
@@ -25,8 +25,9 @@
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -34,12 +35,18 @@
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class PosInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class PosInfIfEqualsDescriptor extends AbstractComparisonDescriptor {
     private static final long serialVersionUID = 1L;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new PosInfIfEqualsDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
         }
     };
 
@@ -55,7 +62,7 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractIfEqualsEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractIfEqualsEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc) {
 
                     final AMutableDouble equalsResult = new AMutableDouble(Double.POSITIVE_INFINITY);
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
index 5f004ef..13acefb 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
@@ -61,6 +61,7 @@
     private final CastTypeEvaluator inputRecordCaster;
     private final CastTypeEvaluator newValueRecordCaster;
     private final SourceLocation sourceLoc;
+    // TODO(ali): switch to ILogicalBinaryComparator
     private final ComparisonHelper comparisonHelper;
 
     RecordReplaceEvaluator(SourceLocation sourceLoc, IScalarEvaluator eval0, IScalarEvaluator eval1,
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 db0046c..d5d23fb 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
@@ -30,6 +30,7 @@
 public class IntervalLogic implements Serializable {
 
     private static final long serialVersionUID = 1L;
+    // TODO(ali): switch to ILogicalBinaryComparator
     private final ComparisonHelper ch;
     private final transient IPointable s1 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable e1 = VoidPointable.FACTORY.createPointable();

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 8
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/238/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6: Code-Review+1

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3145/6/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java:

Line 34:     enum Operation {
minor, but both names seem to be confusing. COMPARE here means EQ only, which is not obvious. ORDER seems to imply that this operation might be used by ORDER BY, but it won't. I'd call them EQ / LT, or EQ / EQ_LT_GT or even have a boolean equalsOnly = true / false. We can address this in the next change though, as this is not critical.


https://asterix-gerrit.ics.uci.edu/#/c/3145/6/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java:

Line 156:             default:
minor. I'd use "case DOUBLE" here instead of 'default' and throw some exception in "default". I know the code above ensures that only these numeric types reach here, but an additional check will make it future-proof. This applies to all other switch statement below.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

(4 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3145/6/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java:

Line 34:     static Result asResult(int result) {
> minor, but both names seem to be confusing. COMPARE here means EQ only, whi
Agreed. I just made it boolean. There has been a debate over both the old and new names.


https://asterix-gerrit.ics.uci.edu/#/c/3145/6/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java:

Line 156:                 return ADoubleSerializerDeserializer.getDouble(b, s);
> +1
Done


Line 156:                 return ADoubleSerializerDeserializer.getDouble(b, s);
> minor. I'd use "case DOUBLE" here instead of 'default' and throw some excep
Good point. Done.


https://asterix-gerrit.ics.uci.edu/#/c/3145/5/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java:

PS5, Line 30: @SuppressWarnings("squid:S1068")
> can we add a comment stating what S1068 is?
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5621/ (11/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5375/ (7/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/441/ (13/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

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

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5053/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3055/ (10/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

(18 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java:

PS1, Line 35: COMPARE,
> COMPARE
Done


PS1, Line 36: ORDER
> ORDER
Done


PS1, Line 39: asResult(int
> "asResult" ? This is basically a cast from int to Result. Could also be a s
Done


PS1, Line 40: return result < 0 ? Result.LT : (result == 0 ? Result.EQ : Result.GT);
            :     }
            : 
            :     Result compare(byte[] leftBytes, int leftStart, int leftLen, byte[] rightBytes, int rightStart, int rightLen)
            :             throws HyracksDataException;
            : 
            :     Resul
> return result < 0 ? Result.LT : (result == 0 ? Result.EQ : Result.GT)
Done


PS1, Line 50: ompare(IAObject lef
> Could this be part of the factory and not a parameter for the comparison?
Done


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java:

Line 61
> just curious. If one is ANY and another is derived then it'll go to Logical
I believe so. I'm checking the runtime tag. If ANY yielded a derived type, it would trigger the complex code. If ANY yielded something else, it would trigger the scalar which should return MISMATCH. I'll include this as one of the test cases later.


Line 137
> we'll need to have a separate method for comparing integer values without c
Done


Line 155
> integer values (tinyint to bigint) should compared as integers (longs) with
Done


PS1, Line 157: 
> Could we switch on the type tag instead of comparing the object(-reference)
Done


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java:

Line 70:     private final IBinaryComparator uuidBinaryComparator =
> may be rename to INEQUALITY_UNDEFINED_TYPES, to make it clear that it's onl
Done


Line 111: 
> unrelated to this particular change. we should use AYearMonthDurationSerial
Done


Line 119:                 break;
> same comment as above. we should use ADayTimeDurationSerializerDeserializer
Done


Line 199:     public Result compare(IAObject leftConstant, byte[] rightBytes, int rightStart, int rightLen) {
> we need to double check whether it can happen or not. If it is not supposed
Ok. I removed it.


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java:

PS1, Line 109:         // both are constants
             :                 return logicalComparator.compare(leftConstant, rightConstant);
             :             } else {
             :                 // left is constant, right isn't
             :                 return logicalComparator.compare(leftConstant, argRight.getByteArray(), argRight.getStartOffset(),
             :                         argRight.getLength());
             :             }
             :         } else {
             :             if (rightConstant != null) {
             :                 // right is constant, left isn't
             :                 return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
             :                         rightConstant);
             :             } else {
             :                 return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
             :          
> if (leftConstant != null) {
Done


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java:

PS1, Line 41:             case MISSING:
            :                 writeMissing(result);
            :                 break;
            :             case NULL:
            :                 writeNull(result);
            :                 break;
            :             case EQ:
            :                 resultStorage.reset();
            :                 writeEqualsResult();
            :                 result.set(resultStorage);
            :                 break;
            :          
> switch(compare()) {
Done.
That's exactly why I had to introduce MISMATCH. Normally, incompatible types would return NULL, but this function returns the left argument if there is a mismatch (of course, it returns the left arg if they are not equal, as well).


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java:

Line 49:         switch (comparisonResult) {
> minor. switch () will be slightly faster (one comparison instead of 3 here)
Done


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java:

PS1, Line 39:     @Override
            :         public IFunctionDescr
> pull-up into superclass?
You mean creating an abstract class?
Done. But not sure if this will make Mike happy now he is dealing with serialization :)
It should be fine, but I'll just loop him in just in case.


PS1, Line 60:         @Override
            :             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
            :                 return new AbstractValueComparisonEvaluator(args[0], leftType, args[1], rightType, ctx, sourceLoc,
            :      
> pull-up into superclass?
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3048/ (5/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/448/ (10/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5020/ (4/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

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

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

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................

[ASTERIXDB-2516][RT] add support for deep comparison 1

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

Details:
Add support for deep comparison.
- add interface ILogicalBinaryComparator
- switch EQ,LT,GT and other functions to use ILogicalBinaryComparator
instead of ComparisonHelper

Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
33 files changed, 1,019 insertions(+), 217 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/45/3145/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3145
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4961/ (2/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

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

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7: Code-Review+1

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

(8 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java:

Line 61:             return new LogicalGenericBinaryComparator(leftType, rightType);
just curious. If one is ANY and another is derived then it'll go to LogicalGenericBinaryComparator. Will it be able to handle this case?


Line 137:     private static double getValue(ATypeTag numericTag, byte[] b, int s) {
we'll need to have a separate method for comparing integer values without converting them to double and use that if both arguments are integers (from tinyint to bigint)


Line 155:     private static double getConstantValue(IAObject numeric) {
integer values (tinyint to bigint) should compared as integers (longs) without converting them to double.


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java:

Line 70:     private static final EnumSet<ATypeTag> UNDEFINED_TYPES =
may be rename to INEQUALITY_UNDEFINED_TYPES, to make it clear that it's only about inequality


Line 111:             case YEARMONTHDURATION:
unrelated to this particular change. we should use AYearMonthDurationSerializerDeserializer.getYearMonth() instead of AInt32SerializerDeserializer.getInt() here. Same for other types: ATimeSerializerDeserializer/ADateSerializerDeserializer.getChronon(),


Line 119:                 result = Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftStart),
same comment as above. we should use ADayTimeDurationSerializerDeserializer.getDayTime(), ADateTimeSerializerDeserializer.getChronon()


Line 199:             // illegal state maybe???
we need to double check whether it can happen or not. If it is not supposed to happen then either throw IllegalState or just remove this if statement and let the next two lines throw NPE.


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java:

Line 49:         if (comparisonResult == Result.MISSING) {
minor. switch () will be slightly faster (one comparison instead of 3 here)


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/170/ (15/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5626/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/463/ (8/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3059/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5033/ (3/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

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

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4966/ (9/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5644/ (10/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7: Code-Review+1

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Analytics Compatibility Compilation Successful
https://goo.gl/18Th8W : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3060/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5380/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5015/ (9/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5012/ (2/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5430/ (11/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

(10 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java:

PS1, Line 35: EQUALITY
COMPARE


PS1, Line 36: INEQUALITY
ORDER


PS1, Line 39: returnResult
"asResult" ? This is basically a cast from int to Result. Could also be a static "of" method in the result enum.


PS1, Line 40: if (result == 0) {
            :             return Result.EQ;
            :         } else if (result < 0) {
            :             return Result.LT;
            :         } else {
            :             return Result.GT;
            :         }
return result < 0 ? Result.LT : (result == 0 ? Result.EQ : Result.GT)


PS1, Line 50: Operation operation
Could this be part of the factory and not a parameter for the comparison?


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/LogicalComparatorUtil.java:

PS1, Line 157: type == BuiltinType.ADOUBLE
Could we switch on the type tag instead of comparing the object(-reference)s?


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java:

PS1, Line 109: if (leftConstant != null && rightConstant != null) {
             :             // both are constants
             :             return logicalComparator.compare(leftConstant, rightConstant, operation);
             :         } else if (leftConstant != null) {
             :             // left is constant, right isn't
             :             return logicalComparator.compare(leftConstant, argRight.getByteArray(), argRight.getStartOffset(),
             :                     argRight.getLength(), operation);
             :         } else if (rightConstant != null) {
             :             // right is constant, left isn't
             :             return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
             :                     rightConstant, operation);
             :         } else {
             :             return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
             :                     argRight.getByteArray(), argRight.getStartOffset(), argRight.getLength(), operation);
             :         }
if (leftConstant != null) {
        if (rightConstant != null) {
            // both are constants
            return logicalComparator.compare(leftConstant, rightConstant, operation);
        } else {
            // left is constant, right isn't
            return logicalComparator.compare(leftConstant, argRight.getByteArray(), argRight.getStartOffset(),
                    argRight.getLength(), operation);
        }
    } else {
        if (rightConstant != null) {
            // right is constant, left isn't
            return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
                    rightConstant, operation);
        } else {
            return logicalComparator.compare(argLeft.getByteArray(), argLeft.getStartOffset(), argLeft.getLength(),
                    argRight.getByteArray(), argRight.getStartOffset(), argRight.getLength(), operation);
        }
    }


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java:

PS1, Line 41:         Result comparisonResult = compare();
            :         if (comparisonResult == Result.MISSING) {
            :             writeMissing(result);
            :         } else if (comparisonResult == Result.NULL) {
            :             writeNull(result);
            :         } else if (comparisonResult == Result.EQ) {
            :             resultStorage.reset();
            :             writeEqualsResult();
            :             result.set(resultStorage);
            :         } else {
            :             result.set(argLeft);
            :         }
switch(compare()) {
        case MISSING:
            writeMissing(result);
            break;
        case NULL:
            writeNull(result);
            break;
        case EQ:
            resultStorage.reset();
            writeEqualsResult();
            result.set(resultStorage);
            break;
        default:
            // do we need to do something about MISMATCH?
            result.set(argLeft);
    }


https://asterix-gerrit.ics.uci.edu/#/c/3145/1/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
File asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java:

PS1, Line 39: private IAType leftType;
            :     private IAType rightType;
pull-up into superclass?


PS1, Line 60: public void setImmutableStates(Object... types) {
            :         leftType = (IAType) types[0];
            :         rightType = (IAType) types[1];
            :     }
pull-up into superclass?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5057/ (6/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5421/ (3/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8998/ (1/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/468/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3079/ (7/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins, Michael Blow, Dmitry Lychagin,

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

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

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................

[ASTERIXDB-2516][RT] add support for deep comparison 1

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

Details:
Add support for deep comparison.
- add interface ILogicalBinaryComparator
- switch EQ,LT,GT and other functions to use ILogicalBinaryComparator
instead of ComparisonHelper

Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
33 files changed, 1,016 insertions(+), 217 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/45/3145/7
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3145
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/468/ (13/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5052/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/211/ (12/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6: Contrib+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/163/ (15/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/439/ (8/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/151/ (15/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

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

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

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................

[ASTERIXDB-2516][RT] add support for deep comparison 1

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

Details:
Add support for deep comparison.
- add interface ILogicalBinaryComparator
- switch EQ,LT,GT and other functions to use ILogicalBinaryComparator
instead of ComparisonHelper

Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
33 files changed, 1,016 insertions(+), 217 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/45/3145/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3145
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/454/ (1/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

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

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Analytics Compatibility Compilation Successful
https://goo.gl/9rLpeK : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/204/ (7/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1: Integration-Tests-1

Integration Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/7864/ : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/450/ (13/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5625/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5014/ (4/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

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

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/160/ (15/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3145/5/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java:

PS5, Line 55: IAType left, IAType right,
these will be null in case of an older (0.9.4) cc; probably these should compare the old way if these are null


https://asterix-gerrit.ics.uci.edu/#/c/3145/5/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java:

PS5, Line 30: @SuppressWarnings("squid:S1068")
can we add a comment stating what S1068 is?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8995/ (9/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/158/ (15/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9018/ (6/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4958/ (6/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

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

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

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................

[ASTERIXDB-2516][RT] add support for deep comparison 1

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

Details:
Add support for deep comparison.
- add interface ILogicalBinaryComparator
- switch EQ,LT,GT and other functions to use ILogicalBinaryComparator
instead of ComparisonHelper

Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/runtime/NullMissingTest.java
M asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
33 files changed, 1,020 insertions(+), 217 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/45/3145/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/3145
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3057/ (5/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/3145/6/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java:

Line 34:     enum Operation {
> minor, but both names seem to be confusing. COMPARE here means EQ only, whi
or COMPARE -> EQUALS, ORDER -> COMPARE


https://asterix-gerrit.ics.uci.edu/#/c/3145/6/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java:

Line 156:             default:
> minor. I'd use "case DOUBLE" here instead of 'default' and throw some excep
+1


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7: Contrib+1 Integration-Tests+1 Verified+1

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5624/ (3/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3145/5/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
File asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java:

PS5, Line 30: @SuppressWarnings("squid:S1068")
> can we add a comment stating what S1068 is?
These are for unused variables. I'll remove them all once I plug in the code for comparing complex types in a subsequent patch.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3061/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/219/ (10/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8999/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8997/ (1/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/448/ (8/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5368/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

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

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/213/ (9/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/232/ (9/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5004/ (10/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5050/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5389/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5428/ (5/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/446/ (13/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/451/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/450/ (11/15)

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

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

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/5435/ (8/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/5631/ (11/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3085/ (14/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9005/ (3/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/4AeiT1 : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

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

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

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................

[ASTERIXDB-2516][RT] add support for deep comparison 1

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

Details:
Add support for deep comparison.
- add interface ILogicalBinaryComparator
- switch EQ,LT,GT and other functions to use ILogicalBinaryComparator
instead of ComparisonHelper

Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
---
M asterixdb/asterix-app/src/test/resources/runtimets/queries/comparison/ComparisonQueries.xml
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp_parser.xml
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/ILogicalBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComparatorUtil.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalComplexBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalGenericBinaryComparator.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/LogicalScalarBinaryComparator.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordReplaceEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
32 files changed, 989 insertions(+), 213 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 7:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 7
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5049/ (7/15)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I623662861e6f3b1fdcff78f8edc0e3216ca10fe1
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2516][RT] add support for deep comparison 1

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

Change subject: [ASTERIXDB-2516][RT] add support for deep comparison 1
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5048/ (7/15)

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

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