You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Ali Alsuliman (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/07/26 09:53:54 UTC

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

Ali Alsuliman has uploaded a new change for review.

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................

[FUN] Use instance comparators for array functions

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

details:
This patch is to make the array functions use instance
comparators instead of a static comparator to avoid
issues caused by parallelism.

Change-Id: Ic03ef8dc1d678d61999917f975bf9d7f301f873f
---
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
6 files changed, 29 insertions(+), 17 deletions(-)


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

diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
index b74ec6a..4ac1f8dc 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.builders.IAsterixListBuilder;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -44,6 +45,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -94,6 +96,7 @@
         private final SourceLocation sourceLoc;
         private final IBinaryHashFunction binaryHashFunction;
         private final Int2ObjectMap<List<IPointable>> hashes;
+        private final IBinaryComparator comp;
         private IPointable item;
         private ArrayBackedValueStorage storage;
 
@@ -102,6 +105,7 @@
             super(args, ctx, inputListType);
             this.sourceLoc = sourceLoc;
             hashes = new Int2ObjectOpenHashMap<>();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             item = pointableAllocator.allocateEmpty();
             storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
             binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
@@ -138,7 +142,7 @@
                         addItem(item, listBuilder, itemInStorage, sameHashes);
                         hashes.put(hash, sameHashes);
                         item = pointableAllocator.allocateEmpty();
-                    } else if (ArrayFunctionsUtil.findItem(item, sameHashes) == null) {
+                    } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
                         // new item, it could happen that two hashes are the same but they are for different items
                         addItem(item, listBuilder, itemInStorage, sameHashes);
                         item = pointableAllocator.allocateEmpty();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
index 1540ecb..706cb53 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
@@ -33,6 +33,7 @@
 import org.apache.asterix.builders.UnorderedListBuilder;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
@@ -56,6 +57,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IMutableValueStorage;
@@ -140,6 +142,7 @@
         private final IObjectPool<List<ValueListIndex>, ATypeTag> arrayListAllocator;
         private final ArrayBackedValueStorage finalResult;
         private final CastTypeEvaluator caster;
+        private final IBinaryComparator comp;
         private IAsterixListBuilder orderedListBuilder;
         private IAsterixListBuilder unorderedListBuilder;
 
@@ -153,6 +156,7 @@
             finalResult = new ArrayBackedValueStorage();
             listAccessor = new ListAccessor();
             caster = new CastTypeEvaluator();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             listsArgs = new IPointable[args.length];
             listsEval = new IScalarEvaluator[args.length];
             for (int i = 0; i < args.length; i++) {
@@ -303,7 +307,7 @@
                 newHashes.add(new ValueListIndex(item, -1));
                 hashes.put(hash, newHashes);
                 return true;
-            } else if (ArrayFunctionsUtil.findItem(item, sameHashes) == null) {
+            } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
                 sameHashes.add(new ValueListIndex(item, -1));
                 return true;
             }
@@ -341,7 +345,7 @@
 
         private void incrementIfExists(List<ValueListIndex> sameHashes, IPointable item, int listIndex,
                 IAsterixListBuilder listBuilder) throws HyracksDataException {
-            ValueListIndex sameValue = ArrayFunctionsUtil.findItem(item, sameHashes);
+            ValueListIndex sameValue = ArrayFunctionsUtil.findItem(item, sameHashes, comp);
             if (sameValue != null && listIndex - sameValue.listIndex == 1) {
                 // found the item, its stamp is OK (stamp saves the last list index that has seen this item)
                 // increment stamp of this item
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
index f99fc4c..12ae2fd 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
@@ -49,7 +49,6 @@
 
 public class ArraySortDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
-    private static final ArraySortComparator COMP = new ArraySortComparator();
     private IAType inputListType;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@@ -83,12 +82,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new ArraySortFunction(args, ctx, sourceLoc);
+                return new ArraySortEval(args, ctx, sourceLoc);
             }
         };
     }
 
-    private static class ArraySortComparator implements Comparator<IPointable> {
+    protected class ArraySortComparator implements Comparator<IPointable> {
         private final IBinaryComparator comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
         @Override
@@ -102,19 +101,19 @@
         }
     }
 
-    public class ArraySortFunction extends AbstractArrayProcessEval {
+    public class ArraySortEval extends AbstractArrayProcessEval {
         private final SourceLocation sourceLoc;
         private final PriorityQueue<IPointable> sortedList;
         private IPointable item;
         private ArrayBackedValueStorage storage;
 
-        public ArraySortFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+        public ArraySortEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
                 throws HyracksDataException {
             super(args, ctx, inputListType);
             this.sourceLoc = sourceLoc;
             item = pointableAllocator.allocateEmpty();
             storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
-            sortedList = new PriorityQueue<>(COMP);
+            sortedList = new PriorityQueue<>(new ArraySortComparator());
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
index 4ff9c30..45b9fdc 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
@@ -24,6 +24,7 @@
 import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 import org.apache.asterix.builders.ArrayListFactory;
 import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
@@ -33,6 +34,7 @@
 import org.apache.asterix.runtime.utils.ArrayFunctionsUtil;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -44,6 +46,7 @@
     private final Int2ObjectMap<List<ValueCounter>> hashes;
     private final IObjectPool<List<ValueCounter>, ATypeTag> arrayListAllocator;
     private final IObjectPool<ValueCounter, ATypeTag> valueCounterAllocator;
+    private final IBinaryComparator comp;
 
     public ArraySymDiffEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLocation,
             IAType[] argTypes) throws HyracksDataException {
@@ -51,6 +54,7 @@
         arrayListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
         valueCounterAllocator = new ListObjectPool<>(new ValueCounterFactory());
         hashes = new Int2ObjectOpenHashMap<>();
+        comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
         binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
                 .createBinaryHashFunction();
     }
@@ -136,7 +140,7 @@
             return true;
         } else {
             // potentially, item already exists
-            ValueCounter itemListIdxCounter = ArrayFunctionsUtil.findItem(item, sameHashes);
+            ValueCounter itemListIdxCounter = ArrayFunctionsUtil.findItem(item, sameHashes, comp);
             if (itemListIdxCounter == null) {
                 // new item
                 addItem(item, listIndex, sameHashes);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
index ccb86c1..0a0a6ef 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
@@ -24,6 +24,7 @@
 import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 import org.apache.asterix.builders.ArrayListFactory;
 import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -41,6 +42,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -106,11 +108,13 @@
         private final IObjectPool<List<IPointable>, ATypeTag> pointableListAllocator;
         private final IBinaryHashFunction binaryHashFunction;
         private final Int2ObjectMap<List<IPointable>> hashes;
+        private final IBinaryComparator comp;
 
         public ArrayUnionEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
             super(args, ctx, true, sourceLoc, argTypes);
             pointableListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
             hashes = new Int2ObjectOpenHashMap<>();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
                     .createBinaryHashFunction();
         }
@@ -142,7 +146,7 @@
                 addItem(listBuilder, item, sameHashes);
                 hashes.put(hash, sameHashes);
                 return true;
-            } else if (ArrayFunctionsUtil.findItem(item, sameHashes) == null) {
+            } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
                 // new item, it could happen that two hashes are the same but they are for different items
                 addItem(listBuilder, item, sameHashes);
                 return true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
index 37ac692a..14f102c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
@@ -20,24 +20,21 @@
 
 import java.util.List;
 
-import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 
 public class ArrayFunctionsUtil {
 
-    private static final IBinaryComparator COMP = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-
     private ArrayFunctionsUtil() {
     }
 
-    public static <T extends IValueReference> T findItem(IValueReference item, List<T> sameHashes)
-            throws HyracksDataException {
+    public static <T extends IValueReference> T findItem(IValueReference item, List<T> sameHashes,
+            IBinaryComparator comp) throws HyracksDataException {
         T sameItem;
         for (int k = 0; k < sameHashes.size(); k++) {
             sameItem = sameHashes.get(k);
-            if (COMP.compare(item.getByteArray(), item.getStartOffset(), item.getLength(), sameItem.getByteArray(),
+            if (comp.compare(item.getByteArray(), item.getStartOffset(), item.getLength(), sameItem.getByteArray(),
                     sameItem.getStartOffset(), sameItem.getLength()) == 0) {
                 return sameItem;
             }

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


[FUN] Use instance comparators for array functions

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

details:
This patch is to make the array functions use instance
comparators instead of a static comparator to avoid
issues caused by parallelism.

Change-Id: Ic03ef8dc1d678d61999917f975bf9d7f301f873f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2816
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
6 files changed, 29 insertions(+), 17 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Jenkins: Verified; No violations found; ; Verified
  Dmitry Lychagin: Looks good to me, approved



diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
index b74ec6a..4ac1f8dc 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayDistinctDescriptor.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.builders.IAsterixListBuilder;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -44,6 +45,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -94,6 +96,7 @@
         private final SourceLocation sourceLoc;
         private final IBinaryHashFunction binaryHashFunction;
         private final Int2ObjectMap<List<IPointable>> hashes;
+        private final IBinaryComparator comp;
         private IPointable item;
         private ArrayBackedValueStorage storage;
 
@@ -102,6 +105,7 @@
             super(args, ctx, inputListType);
             this.sourceLoc = sourceLoc;
             hashes = new Int2ObjectOpenHashMap<>();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             item = pointableAllocator.allocateEmpty();
             storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
             binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
@@ -138,7 +142,7 @@
                         addItem(item, listBuilder, itemInStorage, sameHashes);
                         hashes.put(hash, sameHashes);
                         item = pointableAllocator.allocateEmpty();
-                    } else if (ArrayFunctionsUtil.findItem(item, sameHashes) == null) {
+                    } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
                         // new item, it could happen that two hashes are the same but they are for different items
                         addItem(item, listBuilder, itemInStorage, sameHashes);
                         item = pointableAllocator.allocateEmpty();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
index 1540ecb..706cb53 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayIntersectDescriptor.java
@@ -33,6 +33,7 @@
 import org.apache.asterix.builders.UnorderedListBuilder;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
@@ -56,6 +57,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IMutableValueStorage;
@@ -140,6 +142,7 @@
         private final IObjectPool<List<ValueListIndex>, ATypeTag> arrayListAllocator;
         private final ArrayBackedValueStorage finalResult;
         private final CastTypeEvaluator caster;
+        private final IBinaryComparator comp;
         private IAsterixListBuilder orderedListBuilder;
         private IAsterixListBuilder unorderedListBuilder;
 
@@ -153,6 +156,7 @@
             finalResult = new ArrayBackedValueStorage();
             listAccessor = new ListAccessor();
             caster = new CastTypeEvaluator();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             listsArgs = new IPointable[args.length];
             listsEval = new IScalarEvaluator[args.length];
             for (int i = 0; i < args.length; i++) {
@@ -303,7 +307,7 @@
                 newHashes.add(new ValueListIndex(item, -1));
                 hashes.put(hash, newHashes);
                 return true;
-            } else if (ArrayFunctionsUtil.findItem(item, sameHashes) == null) {
+            } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
                 sameHashes.add(new ValueListIndex(item, -1));
                 return true;
             }
@@ -341,7 +345,7 @@
 
         private void incrementIfExists(List<ValueListIndex> sameHashes, IPointable item, int listIndex,
                 IAsterixListBuilder listBuilder) throws HyracksDataException {
-            ValueListIndex sameValue = ArrayFunctionsUtil.findItem(item, sameHashes);
+            ValueListIndex sameValue = ArrayFunctionsUtil.findItem(item, sameHashes, comp);
             if (sameValue != null && listIndex - sameValue.listIndex == 1) {
                 // found the item, its stamp is OK (stamp saves the last list index that has seen this item)
                 // increment stamp of this item
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
index f99fc4c..12ae2fd 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySortDescriptor.java
@@ -49,7 +49,6 @@
 
 public class ArraySortDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
-    private static final ArraySortComparator COMP = new ArraySortComparator();
     private IAType inputListType;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@@ -83,12 +82,12 @@
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new ArraySortFunction(args, ctx, sourceLoc);
+                return new ArraySortEval(args, ctx, sourceLoc);
             }
         };
     }
 
-    private static class ArraySortComparator implements Comparator<IPointable> {
+    protected class ArraySortComparator implements Comparator<IPointable> {
         private final IBinaryComparator comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
         @Override
@@ -102,19 +101,19 @@
         }
     }
 
-    public class ArraySortFunction extends AbstractArrayProcessEval {
+    public class ArraySortEval extends AbstractArrayProcessEval {
         private final SourceLocation sourceLoc;
         private final PriorityQueue<IPointable> sortedList;
         private IPointable item;
         private ArrayBackedValueStorage storage;
 
-        public ArraySortFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+        public ArraySortEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
                 throws HyracksDataException {
             super(args, ctx, inputListType);
             this.sourceLoc = sourceLoc;
             item = pointableAllocator.allocateEmpty();
             storage = (ArrayBackedValueStorage) storageAllocator.allocate(null);
-            sortedList = new PriorityQueue<>(COMP);
+            sortedList = new PriorityQueue<>(new ArraySortComparator());
         }
 
         @Override
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
index 4ff9c30..45b9fdc 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySymDiffEval.java
@@ -24,6 +24,7 @@
 import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 import org.apache.asterix.builders.ArrayListFactory;
 import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
@@ -33,6 +34,7 @@
 import org.apache.asterix.runtime.utils.ArrayFunctionsUtil;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -44,6 +46,7 @@
     private final Int2ObjectMap<List<ValueCounter>> hashes;
     private final IObjectPool<List<ValueCounter>, ATypeTag> arrayListAllocator;
     private final IObjectPool<ValueCounter, ATypeTag> valueCounterAllocator;
+    private final IBinaryComparator comp;
 
     public ArraySymDiffEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLocation,
             IAType[] argTypes) throws HyracksDataException {
@@ -51,6 +54,7 @@
         arrayListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
         valueCounterAllocator = new ListObjectPool<>(new ValueCounterFactory());
         hashes = new Int2ObjectOpenHashMap<>();
+        comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
         binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
                 .createBinaryHashFunction();
     }
@@ -136,7 +140,7 @@
             return true;
         } else {
             // potentially, item already exists
-            ValueCounter itemListIdxCounter = ArrayFunctionsUtil.findItem(item, sameHashes);
+            ValueCounter itemListIdxCounter = ArrayFunctionsUtil.findItem(item, sameHashes, comp);
             if (itemListIdxCounter == null) {
                 // new item
                 addItem(item, listIndex, sameHashes);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
index ccb86c1..0a0a6ef 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayUnionDescriptor.java
@@ -24,6 +24,7 @@
 import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 import org.apache.asterix.builders.ArrayListFactory;
 import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.formats.nontagged.BinaryHashFunctionFactoryProvider;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
@@ -41,6 +42,7 @@
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IPointable;
@@ -106,11 +108,13 @@
         private final IObjectPool<List<IPointable>, ATypeTag> pointableListAllocator;
         private final IBinaryHashFunction binaryHashFunction;
         private final Int2ObjectMap<List<IPointable>> hashes;
+        private final IBinaryComparator comp;
 
         public ArrayUnionEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
             super(args, ctx, true, sourceLoc, argTypes);
             pointableListAllocator = new ListObjectPool<>(new ArrayListFactory<>());
             hashes = new Int2ObjectOpenHashMap<>();
+            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
             binaryHashFunction = BinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(null)
                     .createBinaryHashFunction();
         }
@@ -142,7 +146,7 @@
                 addItem(listBuilder, item, sameHashes);
                 hashes.put(hash, sameHashes);
                 return true;
-            } else if (ArrayFunctionsUtil.findItem(item, sameHashes) == null) {
+            } else if (ArrayFunctionsUtil.findItem(item, sameHashes, comp) == null) {
                 // new item, it could happen that two hashes are the same but they are for different items
                 addItem(listBuilder, item, sameHashes);
                 return true;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
index 37ac692a..14f102c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/ArrayFunctionsUtil.java
@@ -20,24 +20,21 @@
 
 import java.util.List;
 
-import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IValueReference;
 
 public class ArrayFunctionsUtil {
 
-    private static final IBinaryComparator COMP = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-
     private ArrayFunctionsUtil() {
     }
 
-    public static <T extends IValueReference> T findItem(IValueReference item, List<T> sameHashes)
-            throws HyracksDataException {
+    public static <T extends IValueReference> T findItem(IValueReference item, List<T> sameHashes,
+            IBinaryComparator comp) throws HyracksDataException {
         T sameItem;
         for (int k = 0; k < sameHashes.size(); k++) {
             sameItem = sameHashes.get(k);
-            if (COMP.compare(item.getByteArray(), item.getStartOffset(), item.getLength(), sameItem.getByteArray(),
+            if (comp.compare(item.getByteArray(), item.getStartOffset(), item.getLength(), sameItem.getByteArray(),
                     sameItem.getStartOffset(), sameItem.getLength()) == 0) {
                 return sameItem;
             }

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1: Code-Review+2

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/XQ1GEH : SUCCESS

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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

Change in asterixdb[master]: [FUN] Use instance comparators for array functions

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

Change subject: [FUN] Use instance comparators for array functions
......................................................................


Patch Set 1:

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

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

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