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

Change in asterixdb[master]: WIP. Array comparator. another signature

Till Westmann has uploaded a new change for review.

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

Change subject: WIP. Array comparator. another signature
......................................................................

WIP. Array comparator. another signature

Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper2.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.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/ComparisonHelper.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/NotEqualsDescriptor.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
M hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
23 files changed, 1,547 insertions(+), 122 deletions(-)


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

diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
index 6afc9d8..3b5ae11 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
@@ -29,4 +29,39 @@
     @Override
     public abstract int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) throws HyracksDataException;
 
+    public enum Kind {
+        ORDERING,
+        TOTAL_ORDER,
+        COMPARABLE
+    }
+
+    public enum State {
+        SUCCESS,
+        UNCOMPARABLE,
+        UNSUPPORTED
+    }
+
+    public static class ComparisonState {
+        private State state;
+
+        public ComparisonState() {
+            this(State.SUCCESS);
+        }
+
+        public ComparisonState(State state) {
+            this.state = state;
+        }
+
+        public State getState() {
+            return state;
+        }
+
+        public void setState(State state) {
+            this.state = state;
+        }
+
+        public boolean isSuccess() {
+            return state == State.SUCCESS;
+        }
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
index 99a26ca..4178937 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
@@ -108,6 +108,9 @@
                     APolygonPartialBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             // UUID
             final IBinaryComparator ascUUIDComp = AUUIDPartialBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            // ORDERED LIST
+            final IBinaryComparator orderedListComp =
+                    AOrderedListBinaryComparatorFactory.ORDERING.createBinaryComparator();
             // RAW
             final IBinaryComparator rawComp = RawBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
@@ -319,6 +322,9 @@
                     case BINARY: {
                         return ascByteArrayComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
+                    case ARRAY: {
+                        return orderedListComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    }
                     default: {
                         // We include typeTag in comparison to compare between two type to enforce some ordering
                         return rawComp.compare(b1, s1, l1, b2, s2, l2);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java
new file mode 100644
index 0000000..8953f7e
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java
@@ -0,0 +1,155 @@
+/*
+ * 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.dataflow.data.nontagged.comparators.ABinaryComparator.State;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+public class AOrderedListBinaryComparatorFactory implements IBinaryComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    public final static AOrderedListBinaryComparatorFactory ORDERING =
+            new AOrderedListBinaryComparatorFactory(Kind.ORDERING);
+    public final static AOrderedListBinaryComparatorFactory TOTAL_ORDER =
+            new AOrderedListBinaryComparatorFactory(Kind.TOTAL_ORDER);
+    public final static AOrderedListBinaryComparatorFactory COMPARABLE =
+            new AOrderedListBinaryComparatorFactory(Kind.COMPARABLE);
+
+    private final Kind kind;
+
+    private AOrderedListBinaryComparatorFactory(Kind kind) {
+        this.kind = kind;
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new AOrderedListBinaryComparator(kind);
+    }
+
+    public static class AOrderedListBinaryComparator implements IBinaryComparator {
+
+        private final MutableInt result;
+        private final ComparisonHelper2 comparisonHelper;
+
+        private AOrderedListBinaryComparator(Kind kind) {
+            this.comparisonHelper = new ComparisonHelper2(new SourceLocation(1, 1), kind);
+            this.result = new MutableInt();
+        }
+
+        @Override
+        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) throws HyracksDataException {
+            if (State.SUCCESS == compare(b1, s1, l1, b2, s2, l2, result, comparisonHelper)) {
+                return result.intValue();
+            }
+            throw new IllegalStateException();
+        }
+
+        public static State compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2, MutableInt result,
+                ComparisonHelper2 ch) throws HyracksDataException {
+            ATypeTag declaredItemTag1 = ATypeTag.VALUE_TYPE_MAPPING[b1[s1]];
+            ATypeTag declaredItemTag2 = ATypeTag.VALUE_TYPE_MAPPING[b2[s2]];
+            int numItems1 = AOrderedListSerializerDeserializer.getNumberOfItems(b1, s1, false);
+            int numItems2 = AOrderedListSerializerDeserializer.getNumberOfItems(b2, s2, false);
+
+            int item1Offset;
+            int item2Offset;
+            int item1Length;
+            int item2Length;
+            ATypeTag item1Tag;
+            ATypeTag item2Tag;
+            // TODO: use pool
+            IPointable item1 = new VoidPointable();
+            IPointable item2 = new VoidPointable();
+
+            State state = ABinaryComparator.State.SUCCESS;
+            try {
+                for (int i = 0; i < numItems1 && i < numItems2; i++) {
+                    // get item offset & type
+                    item1Offset = AOrderedListSerializerDeserializer.getItemOffset(b1, s1, i, false);
+                    item2Offset = AOrderedListSerializerDeserializer.getItemOffset(b2, s2, i, false);
+                    if (declaredItemTag1 == ATypeTag.ANY) {
+                        item1Tag = ATypeTag.VALUE_TYPE_MAPPING[b1[item1Offset]];
+                        item1Offset++;
+                    } else {
+                        item1Tag = declaredItemTag1;
+                    }
+
+                    if (declaredItemTag2 == ATypeTag.ANY) {
+                        item2Tag = ATypeTag.VALUE_TYPE_MAPPING[b2[item2Offset]];
+                        item2Offset++;
+                    } else {
+                        item2Tag = declaredItemTag2;
+                    }
+
+                    // get item length
+                    item1Length = NonTaggedFormatUtil.getFieldValueLength(b1, item1Offset, item1Tag, false);
+                    item2Length = NonTaggedFormatUtil.getFieldValueLength(b2, item2Offset, item2Tag, false);
+
+                    item1.set(b1, item1Offset, item1Length);
+                    item2.set(b2, item2Offset, item2Length);
+                    state = ch.compare(item1Tag, item2Tag, item1, item2, result);
+
+                    if (State.SUCCESS != state || result.intValue() != 0) {
+                        return state;
+                    }
+                }
+
+                result.setValue(Integer.compare(numItems1, numItems2));
+                return state;
+            } catch (HyracksDataException e) {
+                throw HyracksDataException.create(e, e.getNodeId());
+            }
+        }
+    }
+
+    @Override
+    public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+        ObjectNode jsonNode = registry.getClassIdentifier(getClass(), serialVersionUID);
+        jsonNode.put("kind", kind.ordinal());
+        return jsonNode;
+    }
+
+    @SuppressWarnings("squid:S1172") // unused parameter
+    public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json) {
+        int kind = json.get("kind").asInt();
+        if (kind == Kind.ORDERING.ordinal()) {
+            return ORDERING;
+        } else if (kind == Kind.TOTAL_ORDER.ordinal()) {
+            return TOTAL_ORDER;
+        } else {
+            return COMPARABLE;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java
new file mode 100644
index 0000000..6ddeef7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java
@@ -0,0 +1,498 @@
+/*
+ * 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 java.io.Serializable;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.ComparisonState;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.State;
+import org.apache.asterix.dataflow.data.nontagged.exceptions.IncompatibleTypeException;
+import org.apache.asterix.dataflow.data.nontagged.exceptions.UnsupportedTypeException;
+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.BinaryComparatorFactoryProvider;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
+
+public class ComparisonHelper implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private static final String COMPARISON = "comparison operations (>, >=, <, and <=)";
+
+    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 IBinaryComparator rawComparator = RawBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+    private final SourceLocation sourceLoc;
+    private final Kind kind;
+
+    public ComparisonHelper(SourceLocation sourceLoc, Kind kind) {
+        this.sourceLoc = sourceLoc;
+        this.kind = kind;
+    }
+
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2,
+            ComparisonState state) throws HyracksDataException {
+        boolean comparable = ATypeHierarchy.isCompatible(typeTag1, typeTag2);
+
+        if (kind == Kind.ORDERING) {
+            if (!comparable) {
+                state.setState(State.SUCCESS);
+                return Integer.compare(typeTag1.serialize(), typeTag2.serialize());
+            } else if (isMultisetOrObject(typeTag1, typeTag2)) {
+                state.setState(State.SUCCESS);
+                return rawComparator.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength(),
+                        arg2.getByteArray(), arg2.getStartOffset(), arg2.getLength());
+            }
+        } else if (!comparable) {
+            state.setState(State.UNCOMPARABLE);
+            return Integer.compare(typeTag1.serialize(), typeTag2.serialize());
+        }
+
+        if (kind == Kind.TOTAL_ORDER) {
+            checkTotalOrderability(typeTag1);
+        }
+
+        switch (typeTag1) {
+            case TINYINT:
+                state.setState(State.SUCCESS);
+                return compareInt8WithArg(typeTag2, arg1, arg2, obj1, obj2);
+            case SMALLINT:
+                state.setState(State.SUCCESS);
+                return compareInt16WithArg(typeTag2, arg1, arg2, obj1, obj2);
+            case INTEGER:
+                state.setState(State.SUCCESS);
+                return compareInt32WithArg(typeTag2, arg1, arg2, obj1, obj2);
+            case BIGINT:
+                state.setState(State.SUCCESS);
+                return compareInt64WithArg(typeTag2, arg1, arg2, obj1, obj2);
+            case FLOAT:
+                state.setState(State.SUCCESS);
+                return compareFloatWithArg(typeTag2, arg1, arg2, obj1, obj2);
+            case DOUBLE:
+                state.setState(State.SUCCESS);
+                return compareDoubleWithArg(typeTag2, arg1, arg2, obj1, obj2);
+            case STRING:
+                return compareStringWithArg(typeTag2, arg1, arg2, state);
+            case BOOLEAN:
+                state.setState(State.SUCCESS);
+                return compareBooleanWithArg(typeTag2, arg1, arg2);
+            default:
+                return compareStrongTypedWithArg(typeTag1, typeTag2, arg1, arg2, state);
+        }
+    }
+
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, ComparisonState result)
+            throws HyracksDataException {
+        return compare(typeTag1, typeTag2, arg1, arg2, null, null, result);
+    }
+
+    private int compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag, IPointable arg1,
+            IPointable arg2, ComparisonState result) throws HyracksDataException {
+        if (expectedTypeTag != actualTypeTag) {
+            throw new IncompatibleTypeException(sourceLoc, COMPARISON, actualTypeTag.serialize(),
+                    expectedTypeTag.serialize());
+        }
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        int leftLen = arg1.getLength() - 1;
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+        int rightLen = arg2.getLength() - 1;
+
+        result.setState(State.SUCCESS);
+        switch (actualTypeTag) {
+            case YEARMONTHDURATION:
+            case TIME:
+            case DATE:
+                return Integer.compare(AInt32SerializerDeserializer.getInt(leftBytes, leftOffset),
+                        AInt32SerializerDeserializer.getInt(rightBytes, rightOffset));
+            case DAYTIMEDURATION:
+            case DATETIME:
+                return Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftOffset),
+                        AInt64SerializerDeserializer.getLong(rightBytes, rightOffset));
+            case CIRCLE:
+                return circleBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+            case LINE:
+                return lineBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+            case POINT:
+                return pointBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+            case POINT3D:
+                return point3DBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen);
+            case POLYGON:
+                return polygonBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen);
+            case DURATION:
+                return durationBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+            case INTERVAL:
+                return intervalBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+            case RECTANGLE:
+                return rectangleBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen);
+            case BINARY:
+                return byteArrayComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+            case UUID:
+                return uuidBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset, rightLen);
+            //case ARRAY:
+            //    return AOrderedListBinaryComparatorFactory.AOrderedListBinaryComparator.compare(leftBytes, leftOffset,
+            //            arg1.getLength(), rightBytes, rightOffset, arg2.getLength(), result, this);
+            default:
+                result.setState(State.UNSUPPORTED);
+                return -1;
+        }
+    }
+
+    private int compareBooleanWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+        if (typeTag2 == ATypeTag.BOOLEAN) {
+            byte b0 = arg1.getByteArray()[arg1.getStartOffset()];
+            byte b1 = arg2.getByteArray()[arg2.getStartOffset()];
+            return compareByte(b0, b1);
+        }
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG,
+                typeTag2.serialize());
+    }
+
+    private int compareStringWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, ComparisonState state)
+            throws HyracksDataException {
+        if (typeTag2 == ATypeTag.STRING) {
+            return strBinaryComp.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength() - 1,
+                    arg2.getByteArray(), arg2.getStartOffset(), arg2.getLength() - 1);
+        }
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_STRING_TYPE_TAG,
+                typeTag2.serialize());
+    }
+
+    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        double s = getOrDeserializeDouble(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareDouble(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareDouble(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareDouble(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareDouble(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareDouble(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default: {
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+                        typeTag2.serialize());
+            }
+        }
+    }
+
+    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        float s = getOrDeserializeFloat(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareFloat(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareFloat(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareFloat(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareFloat(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        long s = getOrDeserializeBigInt(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareLong(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareLong(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareLong(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        int s = getOrDeserializeInt(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareInt(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareInt(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        short s = getOrDeserializeSmallInt(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareShort(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftStart = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightStart = arg2.getStartOffset();
+
+        byte s = getOrDeserializeTinyInt(leftBytes, leftStart, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareByte(s, getOrDeserializeTinyInt(rightBytes, rightStart, obj2));
+            case SMALLINT:
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightStart, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightStart, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightStart, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightStart, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightStart, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private final byte getOrDeserializeTinyInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt8SerializerDeserializer.getByte(bytes, offset) : obj.byteValue();
+    }
+
+    private final short getOrDeserializeSmallInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt16SerializerDeserializer.getShort(bytes, offset) : obj.shortValue();
+    }
+
+    private final int getOrDeserializeInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt32SerializerDeserializer.getInt(bytes, offset) : obj.intValue();
+    }
+
+    private final long getOrDeserializeBigInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt64SerializerDeserializer.getLong(bytes, offset) : obj.longValue();
+    }
+
+    private final float getOrDeserializeFloat(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AFloatSerializerDeserializer.getFloat(bytes, offset) : obj.floatValue();
+    }
+
+    private final double getOrDeserializeDouble(byte[] bytes, int offset, Number obj) {
+        return obj == null ? ADoubleSerializerDeserializer.getDouble(bytes, offset) : obj.doubleValue();
+    }
+
+    private final int compareByte(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareShort(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareInt(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareLong(long v1, long v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareFloat(float v1, float v2) {
+        return Float.compare(v1, v2);
+    }
+
+    private final int compareDouble(double v1, double v2) {
+        return Double.compare(v1, v2);
+    }
+
+    /**
+     * When field value falls into the primitive type groups, we consider to cache its value instead of deserialize it
+     * every time.
+     *
+     * @param bytes
+     * @return primitive value as Number
+     */
+    public Number getNumberValue(byte[] bytes) {
+        ATypeTag aTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[0]);
+        int offset = 1;
+        if (aTypeTag == null) {
+            return null;
+        }
+        switch (aTypeTag) {
+            case TINYINT:
+                return AInt8SerializerDeserializer.getByte(bytes, offset);
+            case SMALLINT:
+                return AInt16SerializerDeserializer.getShort(bytes, offset);
+            case INTEGER:
+                return AInt32SerializerDeserializer.getInt(bytes, offset);
+            case BIGINT:
+                return AInt64SerializerDeserializer.getLong(bytes, offset);
+            case FLOAT:
+                return AFloatSerializerDeserializer.getFloat(bytes, offset);
+            case DOUBLE:
+                return ADoubleSerializerDeserializer.getDouble(bytes, offset);
+            default:
+                return null;
+        }
+    }
+
+    private void checkTotalOrderability(ATypeTag typeTag) throws HyracksDataException {
+        switch (typeTag) {
+            case DURATION:
+            case INTERVAL:
+            case LINE:
+            case POINT:
+            case POINT3D:
+            case POLYGON:
+            case CIRCLE:
+            case RECTANGLE:
+                throw new UnsupportedTypeException(sourceLoc, ComparisonHelper.COMPARISON, typeTag.serialize());
+            default:
+        }
+    }
+
+    private boolean isMultisetOrObject(ATypeTag typeTag1, ATypeTag typeTag2) {
+        return (typeTag1 == ATypeTag.MULTISET && typeTag2 == ATypeTag.MULTISET)
+                || (typeTag1 == ATypeTag.OBJECT && typeTag2 == ATypeTag.OBJECT);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper2.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper2.java
new file mode 100644
index 0000000..5f7c5ae
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper2.java
@@ -0,0 +1,517 @@
+/*
+ * 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 java.io.Serializable;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.ComparisonState;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.State;
+import org.apache.asterix.dataflow.data.nontagged.exceptions.IncompatibleTypeException;
+import org.apache.asterix.dataflow.data.nontagged.exceptions.UnsupportedTypeException;
+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.BinaryComparatorFactoryProvider;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
+
+public class ComparisonHelper2 implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private static final String COMPARISON = "comparison operations (>, >=, <, and <=)";
+
+    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 IBinaryComparator rawComparator = RawBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+    private final SourceLocation sourceLoc;
+    private final Kind kind;
+
+    public ComparisonHelper2(SourceLocation sourceLoc, Kind kind) {
+        this.sourceLoc = sourceLoc;
+        this.kind = kind;
+    }
+
+    public State compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2,
+                         MutableInt result) throws HyracksDataException {
+        boolean comparable = ATypeHierarchy.isCompatible(typeTag1, typeTag2);
+
+        if (kind == Kind.ORDERING) {
+            if (!comparable) {
+                result.setValue(Integer.compare(typeTag1.serialize(), typeTag2.serialize()));
+                return State.SUCCESS;
+            } else if (isMultisetOrObject(typeTag1, typeTag2)) {
+                result.setValue(rawComparator.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength(),
+                        arg2.getByteArray(), arg2.getStartOffset(), arg2.getLength()));
+                return State.SUCCESS;
+            }
+        } else if (!comparable) {
+            result.setValue( Integer.compare(typeTag1.serialize(), typeTag2.serialize()));
+            return State.UNCOMPARABLE;
+        }
+
+        if (kind == Kind.TOTAL_ORDER) {
+            checkTotalOrderability(typeTag1);
+        }
+
+        switch (typeTag1) {
+            case TINYINT:
+                result.setValue(compareInt8WithArg(typeTag2, arg1, arg2, obj1, obj2));
+                return State.SUCCESS;
+            case SMALLINT:
+                result.setValue(compareInt16WithArg(typeTag2, arg1, arg2, obj1, obj2));
+                return State.SUCCESS;
+            case INTEGER:
+                result.setValue(compareInt32WithArg(typeTag2, arg1, arg2, obj1, obj2));
+                return State.SUCCESS;
+            case BIGINT:
+                result.setValue(compareInt64WithArg(typeTag2, arg1, arg2, obj1, obj2));
+                return State.SUCCESS;
+            case FLOAT:
+                result.setValue(compareFloatWithArg(typeTag2, arg1, arg2, obj1, obj2));
+                return State.SUCCESS;
+            case DOUBLE:
+                result.setValue(compareDoubleWithArg(typeTag2, arg1, arg2, obj1, obj2));
+                return State.SUCCESS;
+            case STRING:
+                return compareStringWithArg(typeTag2, arg1, arg2, result);
+            case BOOLEAN:
+                result.setValue(compareBooleanWithArg(typeTag2, arg1, arg2));
+                return State.SUCCESS;
+            default:
+                return compareStrongTypedWithArg(typeTag1, typeTag2, arg1, arg2, result);
+        }
+    }
+
+    public State compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, MutableInt result)
+            throws HyracksDataException {
+        return compare(typeTag1, typeTag2, arg1, arg2, null, null, result);
+    }
+
+    private State compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag, IPointable arg1,
+                                          IPointable arg2, MutableInt result) throws HyracksDataException {
+        if (expectedTypeTag != actualTypeTag) {
+            throw new IncompatibleTypeException(sourceLoc, COMPARISON, actualTypeTag.serialize(),
+                    expectedTypeTag.serialize());
+        }
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        int leftLen = arg1.getLength() - 1;
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+        int rightLen = arg2.getLength() - 1;
+
+        switch (actualTypeTag) {
+            case YEARMONTHDURATION:
+            case TIME:
+            case DATE:
+                result.setValue(Integer.compare(AInt32SerializerDeserializer.getInt(leftBytes, leftOffset),
+                        AInt32SerializerDeserializer.getInt(rightBytes, rightOffset)));
+                return State.SUCCESS;
+            case DAYTIMEDURATION:
+            case DATETIME:
+                result.setValue( Long.compare(AInt64SerializerDeserializer.getLong(leftBytes, leftOffset),
+                        AInt64SerializerDeserializer.getLong(rightBytes, rightOffset)));
+                return State.SUCCESS;
+            case CIRCLE:
+                result.setValue( circleBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen));
+                return State.SUCCESS;
+            case LINE:
+                result.setValue( lineBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes,
+                        rightOffset, rightLen));
+                return State.SUCCESS;
+            case POINT:
+                result.setValue( pointBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes,
+                        rightOffset, rightLen));
+                return State.SUCCESS;
+            case POINT3D:
+                result.setValue( point3DBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen));
+                return State.SUCCESS;
+            case POLYGON:
+                result.setValue( polygonBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen));
+                return State.SUCCESS;
+            case DURATION:
+                result.setValue( durationBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen));
+                return State.SUCCESS;
+            case INTERVAL:
+                result.setValue( intervalBinaryComp.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen));
+                return State.SUCCESS;
+            case RECTANGLE:
+                result.setValue( rectangleBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen));
+                return State.SUCCESS;
+            case BINARY:
+                result.setValue( byteArrayComparator.compare(leftBytes, leftOffset, leftLen, rightBytes, rightOffset,
+                        rightLen));
+                return State.SUCCESS;
+            case UUID:
+                result.setValue( uuidBinaryComparator.compare(leftBytes, leftOffset, leftLen, rightBytes,
+                        rightOffset, rightLen));
+                return State.SUCCESS;
+            case ARRAY:
+                return AOrderedListBinaryComparatorFactory.AOrderedListBinaryComparator.compare(leftBytes, leftOffset,
+                        arg1.getLength(), rightBytes, rightOffset, arg2.getLength(), result, this);
+            default:
+                return State.UNSUPPORTED;
+        }
+    }
+
+    private int compareBooleanWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2) throws HyracksDataException {
+        if (typeTag2 == ATypeTag.BOOLEAN) {
+            byte b0 = arg1.getByteArray()[arg1.getStartOffset()];
+            byte b1 = arg2.getByteArray()[arg2.getStartOffset()];
+            return compareByte(b0, b1);
+        }
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_BOOLEAN_TYPE_TAG,
+                typeTag2.serialize());
+    }
+
+    private State compareStringWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, MutableInt result)
+            throws HyracksDataException {
+        if (typeTag2 == ATypeTag.STRING) {
+            result.setValue(strBinaryComp.compare(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength() - 1,
+                    arg2.getByteArray(), arg2.getStartOffset(), arg2.getLength() - 1));
+            return State.SUCCESS;
+        }
+        throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_STRING_TYPE_TAG,
+                typeTag2.serialize());
+    }
+
+    private int compareDoubleWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        double s = getOrDeserializeDouble(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareDouble(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareDouble(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareDouble(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareDouble(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareDouble(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default: {
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+                        typeTag2.serialize());
+            }
+        }
+    }
+
+    private int compareFloatWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        float s = getOrDeserializeFloat(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareFloat(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareFloat(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareFloat(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareFloat(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_FLOAT_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt64WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        long s = getOrDeserializeBigInt(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareLong(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareLong(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareLong(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt32WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        int s = getOrDeserializeInt(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareInt(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareInt(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT32_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt16WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftOffset = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightOffset = arg2.getStartOffset();
+
+        short s = getOrDeserializeSmallInt(leftBytes, leftOffset, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareShort(s, getOrDeserializeTinyInt(rightBytes, rightOffset, obj2));
+            case SMALLINT:
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightOffset, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightOffset, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightOffset, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightOffset, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightOffset, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private int compareInt8WithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
+            throws HyracksDataException {
+        byte[] leftBytes = arg1.getByteArray();
+        int leftStart = arg1.getStartOffset();
+        byte[] rightBytes = arg2.getByteArray();
+        int rightStart = arg2.getStartOffset();
+
+        byte s = getOrDeserializeTinyInt(leftBytes, leftStart, obj1);
+        switch (typeTag2) {
+            case TINYINT:
+                return compareByte(s, getOrDeserializeTinyInt(rightBytes, rightStart, obj2));
+            case SMALLINT:
+                return compareShort(s, getOrDeserializeSmallInt(rightBytes, rightStart, obj2));
+            case INTEGER:
+                return compareInt(s, getOrDeserializeInt(rightBytes, rightStart, obj2));
+            case BIGINT:
+                return compareLong(s, getOrDeserializeBigInt(rightBytes, rightStart, obj2));
+            case FLOAT:
+                return compareFloat(s, getOrDeserializeFloat(rightBytes, rightStart, obj2));
+            case DOUBLE:
+                return compareDouble(s, getOrDeserializeDouble(rightBytes, rightStart, obj2));
+            default:
+                throw new IncompatibleTypeException(sourceLoc, COMPARISON, ATypeTag.SERIALIZED_INT8_TYPE_TAG,
+                        typeTag2.serialize());
+        }
+    }
+
+    private final byte getOrDeserializeTinyInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt8SerializerDeserializer.getByte(bytes, offset) : obj.byteValue();
+    }
+
+    private final short getOrDeserializeSmallInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt16SerializerDeserializer.getShort(bytes, offset) : obj.shortValue();
+    }
+
+    private final int getOrDeserializeInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt32SerializerDeserializer.getInt(bytes, offset) : obj.intValue();
+    }
+
+    private final long getOrDeserializeBigInt(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AInt64SerializerDeserializer.getLong(bytes, offset) : obj.longValue();
+    }
+
+    private final float getOrDeserializeFloat(byte[] bytes, int offset, Number obj) {
+        return obj == null ? AFloatSerializerDeserializer.getFloat(bytes, offset) : obj.floatValue();
+    }
+
+    private final double getOrDeserializeDouble(byte[] bytes, int offset, Number obj) {
+        return obj == null ? ADoubleSerializerDeserializer.getDouble(bytes, offset) : obj.doubleValue();
+    }
+
+    private final int compareByte(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareShort(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareInt(int v1, int v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareLong(long v1, long v2) {
+        if (v1 == v2) {
+            return 0;
+        } else if (v1 < v2) {
+            return -1;
+        } else {
+            return 1;
+        }
+    }
+
+    private final int compareFloat(float v1, float v2) {
+        return Float.compare(v1, v2);
+    }
+
+    private final int compareDouble(double v1, double v2) {
+        return Double.compare(v1, v2);
+    }
+
+    /**
+     * When field value falls into the primitive type groups, we consider to cache its value instead of deserialize it
+     * every time.
+     *
+     * @param bytes
+     * @return primitive value as Number
+     */
+    public Number getNumberValue(byte[] bytes) {
+        ATypeTag aTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[0]);
+        int offset = 1;
+        if (aTypeTag == null) {
+            return null;
+        }
+        switch (aTypeTag) {
+            case TINYINT:
+                return AInt8SerializerDeserializer.getByte(bytes, offset);
+            case SMALLINT:
+                return AInt16SerializerDeserializer.getShort(bytes, offset);
+            case INTEGER:
+                return AInt32SerializerDeserializer.getInt(bytes, offset);
+            case BIGINT:
+                return AInt64SerializerDeserializer.getLong(bytes, offset);
+            case FLOAT:
+                return AFloatSerializerDeserializer.getFloat(bytes, offset);
+            case DOUBLE:
+                return ADoubleSerializerDeserializer.getDouble(bytes, offset);
+            default:
+                return null;
+        }
+    }
+
+    private void checkTotalOrderability(ATypeTag typeTag) throws HyracksDataException {
+        switch (typeTag) {
+            case DURATION:
+            case INTERVAL:
+            case LINE:
+            case POINT:
+            case POINT3D:
+            case POLYGON:
+            case CIRCLE:
+            case RECTANGLE:
+                throw new UnsupportedTypeException(sourceLoc, ComparisonHelper2.COMPARISON, typeTag.serialize());
+            default:
+        }
+    }
+
+    private boolean isMultisetOrObject(ATypeTag typeTag1, ATypeTag typeTag2) {
+        return (typeTag1 == ATypeTag.MULTISET && typeTag2 == ATypeTag.MULTISET)
+                || (typeTag1 == ATypeTag.OBJECT && typeTag2 == ATypeTag.OBJECT);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
index 2db54c8..6c56449 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
@@ -83,6 +83,8 @@
             final IBinaryComparator ascUUIDComp = AUUIDPartialBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             final IBinaryComparator ascByteArrayComp =
                     new PointableBinaryComparatorFactory(ByteArrayPointable.FACTORY).createBinaryComparator();
+            final IBinaryComparator orderedListComp =
+                    AOrderedListBinaryComparatorFactory.ORDERING.createBinaryComparator();
             final IBinaryComparator rawComp = RawBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
             @Override
@@ -173,6 +175,9 @@
                     case INTERVAL: {
                         return ascIntervalComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
                     }
+                    case ARRAY: {
+                        return orderedListComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
+                    }
                     case BINARY: {
                         return ascByteArrayComp.compare(b1, s1 + skip1, l1 - skip1, b2, s2 + skip2, l2 - skip2);
                     }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java
new file mode 100644
index 0000000..7bcc48c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java
@@ -0,0 +1,45 @@
+/*
+ *  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.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class IncompatibleTypeException extends RuntimeDataException {
+
+    // Incompatible input parameters, e.g., "1.0" > 1.0
+    public IncompatibleTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte typeTagLeft,
+            byte typeTagRight) {
+        super(ErrorCode.TYPE_INCOMPATIBLE, sourceLoc, fid.getName(),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
+    }
+
+    // Incompatible input parameters, e.g., "1.0" > 1.0
+    public IncompatibleTypeException(SourceLocation sourceLoc, String functionName, byte typeTagLeft,
+            byte typeTagRight) {
+        super(ErrorCode.TYPE_INCOMPATIBLE, sourceLoc, functionName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagLeft),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(typeTagRight));
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.java
new file mode 100644
index 0000000..9811493
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.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.nontagged.exceptions;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class UnsupportedTypeException extends RuntimeDataException {
+
+    // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, FunctionIdentifier fid, byte actualTypeTag) {
+        super(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, fid.getName(),
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+
+    // Unsupported input type.
+    public UnsupportedTypeException(String funcName, byte actualTypeTag) {
+        super(ErrorCode.TYPE_UNSUPPORTED, funcName, EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+
+    // Unsupported input type.
+    public UnsupportedTypeException(SourceLocation sourceLoc, String funcName, byte actualTypeTag) {
+        super(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, funcName,
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
index 9307627..d51f93b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
@@ -42,6 +42,7 @@
 public class AOrderedListSerializerDeserializer implements ISerializerDeserializer<AOrderedList> {
 
     private static final long serialVersionUID = 1L;
+    private static final int HEADER = 10; // 10 = tag (1) + itemTag (1) + list size (4) + number of items (4)
     public static final AOrderedListSerializerDeserializer SCHEMALESS_INSTANCE =
             new AOrderedListSerializerDeserializer();
 
@@ -116,32 +117,54 @@
         listBuilder.write(out, false);
     }
 
-    public static final int getOrderedListLength(byte[] serOrderedList, int offset) {
+    public static int getOrderedListLength(byte[] serOrderedList, int offset) {
         return AInt32SerializerDeserializer.getInt(serOrderedList, offset + 1);
     }
 
     public static int getNumberOfItems(byte[] serOrderedList, int offset) {
-        if (serOrderedList[offset] == ATypeTag.ARRAY.serialize()) {
-            // 6 = tag (1) + itemTag (1) + list size (4)
-            return AInt32SerializerDeserializer.getInt(serOrderedList, offset + 6);
-        } else {
-            return -1;
-        }
+        return getNumberOfItems(serOrderedList, offset, true);
     }
 
     public static int getItemOffset(byte[] serOrderedList, int offset, int itemIndex) throws HyracksDataException {
-        if (serOrderedList[offset] == ATypeTag.ARRAY.serialize()) {
-            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[offset + 1]);
-            if (NonTaggedFormatUtil.isFixedSizedCollection(typeTag)) {
-                int length = NonTaggedFormatUtil.getFieldValueLength(serOrderedList, offset + 1, typeTag, true);
-                return offset + 10 + (length * itemIndex);
-            } else {
-                return offset + AInt32SerializerDeserializer.getInt(serOrderedList, offset + 10 + (4 * itemIndex));
-            }
-            // 10 = tag (1) + itemTag (1) + list size (4) + number of items (4)
+        return getItemOffset(serOrderedList, offset, itemIndex, true);
+    }
+
+    public static int getNumberOfItems(byte[] serOrderedList, int offset, boolean tagged) {
+        if (tagged && serOrderedList[offset] == ATypeTag.ARRAY.serialize()) {
+            // 6 = tag (1) + itemTag (1) + list size (4)
+            return AInt32SerializerDeserializer.getInt(serOrderedList, offset + 6);
+        } else if (!tagged) {
+            // 5 = itemTag (1) + list size (4)
+            return AInt32SerializerDeserializer.getInt(serOrderedList, offset + 5);
         } else {
             return -1;
         }
     }
 
+    public static int getItemOffset(byte[] serOrderedList, int offset, int itemIndex, boolean tagged)
+            throws HyracksDataException {
+        int itemTagOffset;
+        int header;
+        int adjustLength;
+        if (tagged && serOrderedList[offset] == ATypeTag.ARRAY.serialize()) {
+            itemTagOffset = 1;
+            header = HEADER;
+            adjustLength = 0;
+        } else if (!tagged) {
+            itemTagOffset = 0;
+            header = HEADER - 1;
+            adjustLength = 1;
+        } else {
+            return -1;
+        }
+
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[offset + itemTagOffset]);
+        if (NonTaggedFormatUtil.isFixedSizedCollection(typeTag)) {
+            int length = NonTaggedFormatUtil.getFieldValueLength(serOrderedList, offset + itemTagOffset, typeTag, true);
+            return offset + header + (length * itemIndex);
+        } else {
+            return offset + AInt32SerializerDeserializer.getInt(serOrderedList, offset + header + (4 * itemIndex))
+                    - adjustLength;
+        }
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java
index 297b4c6..57ca780 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.dataflow.data.nontagged.comparators.ALinePartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectDescBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AOrderedListBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.APoint3DPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.APointPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.APolygonPartialBinaryComparatorFactory;
@@ -151,6 +152,8 @@
                 return addOffset(intervalBinaryComparatorFactory(ascending), ascending);
             case UUID:
                 return addOffset(AUUIDPartialBinaryComparatorFactory.INSTANCE, ascending);
+            case ARRAY:
+                return addOffset(AOrderedListBinaryComparatorFactory.ORDERING, ascending);
             case BINARY:
                 return addOffset(BINARY_POINTABLE_INSTANCE, ascending);
             default:
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..85669b6 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
@@ -18,11 +18,15 @@
  */
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import static org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.State;
+
 import java.io.DataOutput;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.commons.lang3.mutable.MutableInt;
 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,17 +50,17 @@
     protected final IScalarEvaluator evalLeft;
     protected final IScalarEvaluator evalRight;
     protected final SourceLocation sourceLoc;
-    private final ComparisonHelper ch;
+    private final org.apache.asterix.dataflow.data.nontagged.comparators.ComparisonHelper2 ch;
     private Number leftValue;
     private Number rightValue;
 
     public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc, Kind compKind)
             throws HyracksDataException {
         this.evalLeft = evalLeftFactory.createScalarEvaluator(ctx);
         this.evalRight = evalRightFactory.createScalarEvaluator(ctx);
         this.sourceLoc = sourceLoc;
-        ch = new ComparisonHelper(sourceLoc);
+        ch = new org.apache.asterix.dataflow.data.nontagged.comparators.ComparisonHelper2(sourceLoc, compKind);
         leftValue = getValueOfConstantEval(evalLeftFactory);
         rightValue = getValueOfConstantEval(evalRightFactory);
     }
@@ -91,9 +95,9 @@
         return ATypeHierarchy.isCompatible(typeTag1, typeTag2);
     }
 
-    int compare() throws HyracksDataException {
+    State compare(MutableInt result) 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);
+        return ch.compare(leftTypeTag, rightTypeTag, outLeft, outRight, leftValue, rightValue, result);
     }
 }
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..7872026 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,10 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.ComparisonState;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
+import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -27,20 +31,24 @@
 
 public abstract class AbstractIfEqualsEvaluator extends AbstractComparisonEvaluator {
 
+    private final MutableInt res = new MutableInt();
+
     AbstractIfEqualsEvaluator(IScalarEvaluatorFactory evalLeftFactory, IScalarEvaluatorFactory evalRightFactory,
             IHyracksTaskContext ctx, SourceLocation sourceLoc) throws HyracksDataException {
-        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc, Kind.COMPARABLE);
     }
 
     @Override
     protected void evaluateImpl(IPointable result) throws HyracksDataException {
-        if (comparabilityCheck() && compare() == 0) {
-            resultStorage.reset();
-            writeEqualsResult();
-            result.set(resultStorage);
-        } else {
-            result.set(argLeft);
+        if (comparabilityCheck()) {
+            if (compare(res) == ABinaryComparator.State.SUCCESS && res.intValue() == 0) {
+                resultStorage.reset();
+                writeEqualsResult();
+                result.set(resultStorage);
+                return;
+            }
         }
+        result.set(argLeft);
     }
 
     protected abstract void writeEqualsResult() throws HyracksDataException;
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..b637ee3 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,14 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import static org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.State;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 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.commons.lang3.mutable.MutableInt;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -41,10 +42,13 @@
     protected ISerializerDeserializer<ANull> nullSerde =
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
 
+    private final MutableInt res = new MutableInt();
+
     public AbstractValueComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
-            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext ctx, SourceLocation sourceLoc, Kind compKind)
             throws HyracksDataException {
-        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc);
+        super(evalLeftFactory, evalRightFactory, ctx, sourceLoc, compKind);
+
     }
 
     @Override
@@ -53,16 +57,18 @@
 
         // checks whether we can apply >, >=, <, and <= to the given type since
         // these operations cannot be defined for certain types.
-        if (isTotallyOrderable()) {
-            checkTotallyOrderable();
-        }
+        //        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);
+            if (compare(res) == State.SUCCESS) {
+                serde.serialize(ABoolean.valueOf(getComparisonResult(res.intValue())), out);
+            } else {
+                nullSerde.serialize(ANull.NULL, out);
+            }
         } else {
             // result:NULL - two types cannot be compared.
             nullSerde.serialize(ANull.NULL, out);
@@ -70,29 +76,29 @@
         result.set(resultStorage);
     }
 
-    protected abstract boolean isTotallyOrderable();
+    //    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;
-            }
-        }
-    }
+    //    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;
+    //            }
+    //        }
+    //    }
 
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
index 96c5250..ff1879d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/ComparisonHelper.java
@@ -20,6 +20,7 @@
 
 import java.io.Serializable;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.ComparisonState;
 import org.apache.asterix.dataflow.data.nontagged.comparators.ACirclePartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.ADurationPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.AIntervalAscPartialBinaryComparatorFactory;
@@ -46,8 +47,6 @@
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
-
-import static org.apache.asterix.om.types.ATypeTag.TINYINT;
 
 public class ComparisonHelper implements Serializable {
     private static final long serialVersionUID = 1L;
@@ -82,8 +81,8 @@
         this.sourceLoc = sourceLoc;
     }
 
-    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2)
-            throws HyracksDataException {
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, Number obj1, Number obj2,
+            ComparisonState r) throws HyracksDataException {
         switch (typeTag1) {
             case TINYINT:
                 return compareInt8WithArg(typeTag2, arg1, arg2, obj1, obj2);
@@ -102,17 +101,17 @@
             case BOOLEAN:
                 return compareBooleanWithArg(typeTag2, arg1, arg2);
             default:
-                return compareStrongTypedWithArg(typeTag1, typeTag2, arg1, arg2);
+                return compareStrongTypedWithArg(typeTag1, typeTag2, arg1, arg2, r);
         }
     }
 
-    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2)
+    public int compare(ATypeTag typeTag1, ATypeTag typeTag2, IPointable arg1, IPointable arg2, ComparisonState r)
             throws HyracksDataException {
-        return compare(typeTag1, typeTag2, arg1, arg2, null, null);
+        return compare(typeTag1, typeTag2, arg1, arg2, null, null, r);
     }
 
     private int compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag, IPointable arg1,
-            IPointable arg2) throws HyracksDataException {
+            IPointable arg2, ComparisonState r) throws HyracksDataException {
         if (expectedTypeTag != actualTypeTag) {
             throw new IncompatibleTypeException(sourceLoc, COMPARISON, actualTypeTag.serialize(),
                     expectedTypeTag.serialize());
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..8b184ff 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,6 +19,7 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,17 +51,17 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc, Kind.COMPARABLE) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
                         return r == 0;
                     }
 
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return false;
-                    }
+                    //                    @Override
+                    //                    protected boolean isTotallyOrderable() {
+                    //                        return false;
+                    //                    }
                 };
             }
 
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..239fece 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,6 +19,7 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,17 +51,17 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc, Kind.TOTAL_ORDER) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
                         return r > 0;
                     }
 
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
-                    }
+                    //                    @Override
+                    //                    protected boolean isTotallyOrderable() {
+                    //                        return true;
+                    //                    }
                 };
             }
 
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..c0f2a74 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,6 +19,7 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,17 +51,17 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc, Kind.TOTAL_ORDER) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
                         return r >= 0;
                     }
 
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
-                    }
+                    //                    @Override
+                    //                    protected boolean isTotallyOrderable() {
+                    //                        return true;
+                    //                    }
                 };
             }
 
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..76fbb0a 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,6 +19,7 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,17 +51,17 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc, Kind.TOTAL_ORDER) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
                         return r < 0;
                     }
 
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
-                    }
+                    //                    @Override
+                    //                    protected boolean isTotallyOrderable() {
+                    //                        return true;
+                    //                    }
                 };
             }
         };
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..2fdb642 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,6 +19,7 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,17 +51,17 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc, Kind.TOTAL_ORDER) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
                         return r <= 0;
                     }
 
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return true;
-                    }
+                    //                    @Override
+                    //                    protected boolean isTotallyOrderable() {
+                    //                        return true;
+                    //                    }
                 };
             }
 
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..74619dc 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,6 +19,7 @@
 
 package org.apache.asterix.runtime.evaluators.comparisons;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.Kind;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -50,17 +51,17 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc) {
+                return new AbstractValueComparisonEvaluator(args[0], args[1], ctx, sourceLoc, Kind.COMPARABLE) {
 
                     @Override
                     protected boolean getComparisonResult(int r) {
                         return r != 0;
                     }
 
-                    @Override
-                    protected boolean isTotallyOrderable() {
-                        return false;
-                    }
+                    //                    @Override
+                    //                    protected boolean isTotallyOrderable() {
+                    //                        return false;
+                    //                    }
                 };
             }
 
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..2e9cb7f 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
@@ -23,7 +23,11 @@
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator.ComparisonState;
 import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ComparisonHelper;
+import org.apache.asterix.dataflow.data.nontagged.exceptions.UnsupportedTypeException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.pointables.ARecordVisitablePointable;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -32,7 +36,6 @@
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.comparisons.ComparisonHelper;
 import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
@@ -62,6 +65,7 @@
     private final CastTypeEvaluator newValueRecordCaster;
     private final SourceLocation sourceLoc;
     private final ComparisonHelper comparisonHelper;
+    private final ComparisonState state;
 
     RecordReplaceEvaluator(SourceLocation sourceLoc, IScalarEvaluator eval0, IScalarEvaluator eval1,
             IScalarEvaluator eval2, IAType[] argTypes) {
@@ -72,7 +76,8 @@
         openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
         inputRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[0], eval0);
         newValueRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[2], eval2);
-        comparisonHelper = new ComparisonHelper(sourceLoc);
+        comparisonHelper = new ComparisonHelper(sourceLoc, ABinaryComparator.Kind.COMPARABLE);
+        state = new ComparisonState();
     }
 
     @Override
@@ -103,11 +108,11 @@
             newValueRecordCaster.evaluate(tuple, newValuePointable);
         }
         resultStorage.reset();
-        buildOutputRecord(oldValueType);
+        buildOutputRecord(oldValueType, state);
         result.set(resultStorage);
     }
 
-    private void buildOutputRecord(ATypeTag oldValueTypeTag) throws HyracksDataException {
+    private void buildOutputRecord(ATypeTag oldValueTypeTag, ComparisonState state) throws HyracksDataException {
         openRecordPointable.set(inputRecordPointable);
         outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
         outRecordBuilder.init();
@@ -117,7 +122,7 @@
             final IVisitablePointable fieldName = fieldNames.get(i);
             final IVisitablePointable fieldValue = fieldValues.get(i);
             final ATypeTag existingValueTypeTag = PointableHelper.getTypeTag(fieldValue);
-            if (isEqual(existingValueTypeTag, fieldValue, oldValueTypeTag, oldValuePointable)) {
+            if (isEqual(existingValueTypeTag, fieldValue, oldValueTypeTag, oldValuePointable, state)) {
                 outRecordBuilder.addField(fieldName, newValuePointable);
             } else {
                 outRecordBuilder.addField(fieldName, fieldValue);
@@ -143,14 +148,19 @@
         }
     }
 
-    private boolean isEqual(ATypeTag typeTag1, IPointable value1, ATypeTag typeTag2, IPointable value2)
-            throws HyracksDataException {
+    private boolean isEqual(ATypeTag typeTag1, IPointable value1, ATypeTag typeTag2, IPointable value2,
+            ComparisonState state) throws HyracksDataException {
         if (!ATypeHierarchy.isCompatible(typeTag1, typeTag2)) {
             return false;
         }
         setValuePointer(value1, existingValuePtr);
         setValuePointer(value2, oldValuePtr);
-        return comparisonHelper.compare(typeTag1, typeTag2, existingValuePtr, oldValuePtr) == 0;
+        int result = comparisonHelper.compare(typeTag1, typeTag2, existingValuePtr, oldValuePtr, state);
+        if (state.getState() == ABinaryComparator.State.UNSUPPORTED) {
+            // TODO: put something useful
+            throw new UnsupportedTypeException(sourceLoc, "Function name", typeTag1.serialize());
+        }
+        return result == 0 && state.isSuccess();
     }
 
     private static void setValuePointer(IPointable src, IPointable value) {
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..f77ba26 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
@@ -20,8 +20,10 @@
 
 import java.io.Serializable;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ABinaryComparator;
+import org.apache.asterix.dataflow.data.nontagged.comparators.ComparisonHelper;
 import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
-import org.apache.asterix.runtime.evaluators.comparisons.ComparisonHelper;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -35,15 +37,21 @@
     private final transient IPointable e1 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable s2 = VoidPointable.FACTORY.createPointable();
     private final transient IPointable e2 = VoidPointable.FACTORY.createPointable();
+    private final ABinaryComparator.ComparisonState state = new ABinaryComparator.ComparisonState();
 
     public IntervalLogic(SourceLocation sourceLoc) {
-        ch = new ComparisonHelper(sourceLoc);
+        ch = new ComparisonHelper(sourceLoc, ABinaryComparator.Kind.COMPARABLE);
     }
 
     public boolean validateInterval(AIntervalPointable ip1) throws HyracksDataException {
         ip1.getStart(s1);
         ip1.getEnd(e1);
-        return ch.compare(ip1.getTypeTag(), ip1.getTypeTag(), s1, e1) <= 0;
+        int result = ch.compare(ip1.getTypeTag(), ip1.getTypeTag(), s1, e1, state);
+        if (!state.isSuccess()) {
+            // TODO: put something good
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result <= 0;
     }
 
     /**
@@ -58,7 +66,11 @@
     public boolean before(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
         ip1.getEnd(e1);
         ip2.getStart(s2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) < 0;
+        int result = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result < 0;
     }
 
     public boolean after(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
@@ -77,7 +89,11 @@
     public boolean meets(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
         ip1.getEnd(e1);
         ip2.getStart(s2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) == 0;
+        int result = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result == 0;
     }
 
     public boolean metBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
@@ -98,9 +114,19 @@
         ip1.getEnd(e1);
         ip2.getStart(s2);
         ip2.getEnd(e2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2) < 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) > 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) < 0;
+        int result1 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        int result2 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        int result3 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result1 < 0 && result2 > 0 && result3 < 0;
     }
 
     public boolean overlappedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
@@ -120,8 +146,15 @@
         ip1.getEnd(e1);
         ip2.getStart(s2);
         ip2.getEnd(e2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, e2) < 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2) > 0;
+        int result1 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, e2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        int result2 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result1 < 0 && result2 > 0;
     }
 
     /**
@@ -138,8 +171,12 @@
         ip1.getEnd(e1);
         ip2.getStart(s2);
         ip2.getEnd(e2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2) == 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) <= 0;
+        int result1 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        int result2 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2, state);
+        return result1 == 0 && result2 <= 0;
     }
 
     public boolean startedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
@@ -160,8 +197,15 @@
         ip1.getEnd(e1);
         ip2.getStart(s2);
         ip2.getEnd(e2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2) <= 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) >= 0;
+        int result1 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        int result2 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result1 <= 0 && result2 >= 0;
     }
 
     public boolean coveredBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
@@ -182,8 +226,15 @@
         ip1.getEnd(e1);
         ip2.getStart(s2);
         ip2.getEnd(e2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2) >= 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) == 0;
+        int result1 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        int result2 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result1 >= 0 && result2 == 0;
     }
 
     public boolean endedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
@@ -203,8 +254,15 @@
         ip1.getEnd(e1);
         ip2.getStart(s2);
         ip2.getEnd(e2);
-        return ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2) == 0
-                && ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2) == 0;
+        int result1 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), s1, s2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        int result2 = ch.compare(ip1.getTypeTag(), ip2.getTypeTag(), e1, e2, state);
+        if (!state.isSuccess()) {
+            throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.TYPE_INCOMPATIBLE, "Comparison Error");
+        }
+        return result1 == 0 && result2 == 0;
     }
 
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
index 01acddd..23b33f5 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
@@ -21,6 +21,7 @@
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 
+// TODO: this is duplicate
 public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
 
     private static final long serialVersionUID = 1L;

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4943/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/591/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

Posted by "Till Westmann (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/2919

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

Change subject: WIP. Array comparator. another signature
......................................................................

WIP. Array comparator. another signature

Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.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/ComparisonHelper.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/NotEqualsDescriptor.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
M hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
22 files changed, 1,002 insertions(+), 111 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/2919/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2919
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/593/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4299/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has uploaded a new patch set (#2).

Change subject: WIP. Array comparator. another signature
......................................................................

WIP. Array comparator. another signature

Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper2.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.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/ComparisonHelper.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/NotEqualsDescriptor.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
M hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
23 files changed, 1,540 insertions(+), 117 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4389/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4385/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/664/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4296/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4390/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/669/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8326/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4392/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/665/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4390/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

Posted by "Till Westmann (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/2919

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

Change subject: WIP. Array comparator. another signature
......................................................................

WIP. Array comparator. another signature

Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.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/ComparisonHelper.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/NotEqualsDescriptor.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
M hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
22 files changed, 1,007 insertions(+), 111 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/19/2919/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2919
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4388/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/4776/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Analytics Compatibility Compilation Successful
https://goo.gl/22afqJ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/667/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4297/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9861/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8327/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9867/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4946/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4301/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/590/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2401/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/4300/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/668/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2397/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/595/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/594/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

Posted by "Till Westmann (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/2919

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

Change subject: WIP. Array comparator. another signature
......................................................................

WIP. Array comparator. another signature

Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
---
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ABinaryComparator.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AOrderedListBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ComparisonHelper.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/ListItemBinaryComparatorFactory.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/IncompatibleTypeException.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/exceptions/UnsupportedTypeException.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AOrderedListSerializerDeserializer.java
M asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryComparatorFactoryProvider.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/ComparisonHelper.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/NotEqualsDescriptor.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
M hyracks-fullstack/hyracks/hyracks-hdfs/src/main/java/org/apache/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
22 files changed, 1,000 insertions(+), 111 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4942/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/8325/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4387/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 5:

WARNING: THIS CHANGE CONTAINS CROSS-PRODUCT CHANGES IN:
* asterixdb
* hyracks-fullstack

PLEASE REVIEW CAREFULLY AND LOOK FOR API CHANGES!

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: WIP. Array comparator. another signature

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

Change subject: WIP. Array comparator. another signature
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4391/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8d7ac25a62e20e03d205dda76bb469c906093dce
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No