You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by ti...@apache.org on 2018/06/28 12:40:15 UTC

asterixdb git commit: [NO ISSUE][FUN] Implement array_contains() function

Repository: asterixdb
Updated Branches:
  refs/heads/master d954c47c1 -> 9c9ed8058


[NO ISSUE][FUN] Implement array_contains() function

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

details:
This is part of implementing array functions.
The array_contains() takes an input list and a value
and returns true if the value is present in the list.
array_contains(list, val). An error is thrown if
val is object or list.

Change-Id: Ib0222ebdb4dcaaab696ebe23973be8b2347da64b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2729
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: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: 9c9ed805899e4d9884f4e260c53d8ddd173d870a
Parents: d954c47
Author: Ali Alsuliman <al...@gmail.com>
Authored: Wed Jun 27 22:16:19 2018 -0700
Committer: Till Westmann <ti...@apache.org>
Committed: Thu Jun 28 05:39:19 2018 -0700

----------------------------------------------------------------------
 .../array_contains/array_contains.1.ddl.sqlpp   |  46 ++++++++
 .../array_contains.2.update.sqlpp               |  22 ++++
 .../array_contains/array_contains.3.query.sqlpp |  34 ++++++
 .../array_contains/array_contains.4.query.sqlpp |  22 ++++
 .../array_contains/array_contains.5.query.sqlpp |  22 ++++
 .../array_contains/array_contains.6.ddl.sqlpp   |  20 ++++
 .../array_contains/array_contains.3.adm         |   1 +
 .../resources/runtimets/testsuite_sqlpp.xml     |   7 ++
 .../asterix/om/functions/BuiltinFunctions.java  |   3 +
 .../functions/AbstractArraySearchEval.java      | 109 +++++++++++++++++++
 .../functions/ArrayContainsDescriptor.java      |  83 ++++++++++++++
 .../functions/ArrayPositionDescriptor.java      |  71 +-----------
 .../runtime/functions/FunctionCollection.java   |   2 +
 13 files changed, 374 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.1.ddl.sqlpp
new file mode 100755
index 0000000..257c4dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.1.ddl.sqlpp
@@ -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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.TwitterUserType as
+{
+  `screen-name` : string,
+  lang : string,
+  friends_count : bigint,
+  statuses_count : bigint,
+  name : string,
+  followers_count : bigint
+};
+
+create type TinySocial.TweetMessageType as
+ closed {
+  tweetid : string,
+  user : TwitterUserType,
+  `sender-location` : point?,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.2.update.sqlpp
new file mode 100755
index 0000000..4a0e7ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+load  dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.3.query.sqlpp
new file mode 100755
index 0000000..0441a31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.3.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+{
+  "t1": (select array_contains(t.`referred-topics`, "speed") from TweetMessages t order by t.tweetid),
+  "t2": (select array_contains([3,8,98,40], 8)),
+  "t3": (select array_contains([3,8,98,40], 40.0)),
+  "t4": (select array_contains([3,8,98,40], -3)),
+  "t5": (select array_contains([3,"sth",98,40], 98)),
+  "t6": (select array_contains([3,8,98,40], null)),
+  "t7": (select array_contains([3,8,98,40], missing)),
+  "t8": (select array_contains(missing, 6)),
+  "t9": (select array_contains(null, 6)),
+  "t10": (select array_contains(5, "sth")),
+  "t11": (select array_contains([5, {"id":77}, "sth"], "sth"))
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.4.query.sqlpp
new file mode 100755
index 0000000..0cdd2e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+select array_contains([5,1,9], [2,3]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.5.query.sqlpp
new file mode 100755
index 0000000..ee9d988
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.5.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+use TinySocial;
+
+select array_contains([5,{"id": 5},9], {"id": 5});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.6.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.6.ddl.sqlpp
new file mode 100644
index 0000000..3f8c8ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_contains/array_contains.6.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+drop  dataverse TinySocial;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_contains/array_contains.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_contains/array_contains.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_contains/array_contains.3.adm
new file mode 100644
index 0000000..9c1f2d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_contains/array_contains.3.adm
@@ -0,0 +1 @@
+{ "t1": [ { "$1": false }, { "$1": false }, { "$1": false }, { "$1": false }, { "$1": false }, { "$1": true }, { "$1": false }, { "$1": true }, { "$1": false }, { "$1": false }, { "$1": false }, { "$1": false } ], "t2": [ { "$2": true } ], "t3": [ { "$3": true } ], "t4": [ { "$4": false } ], "t5": [ { "$5": true } ], "t6": [ { "$6": null } ], "t7": [ {  } ], "t8": [ {  } ], "t9": [ { "$9": null } ], "t10": [ { "$10": null } ], "t11": [ { "$11": true } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 14b4cc1..9622376 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -1000,6 +1000,13 @@
         <output-dir compare="Text">array_reverse</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_contains">
+        <output-dir compare="Text">array_contains</output-dir>
+        <expected-error>HYR0115: Cannot compare non-primitive values (in line 22, at column 8)</expected-error>
+        <expected-error>HYR0115: Cannot compare non-primitive values (in line 22, at column 8)</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="boolean">
     <test-case FilePath="boolean">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 1570cab..c2b0c02 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -190,6 +190,8 @@ public class BuiltinFunctions {
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-position", 2);
     public static final FunctionIdentifier ARRAY_REVERSE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-reverse", 1);
+    public static final FunctionIdentifier ARRAY_CONTAINS =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-contains", 2);
 
     // objects
     public static final FunctionIdentifier RECORD_MERGE =
@@ -1468,6 +1470,7 @@ public class BuiltinFunctions {
         addFunction(ARRAY_APPEND, ArrayAppendTypeComputer.INSTANCE, true);
         addFunction(ARRAY_POSITION, AInt32TypeComputer.INSTANCE, true);
         addFunction(ARRAY_REVERSE, AListTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_CONTAINS, ABooleanTypeComputer.INSTANCE, true);
 
         // objects
         addFunction(RECORD_MERGE, RecordMergeTypeComputer.INSTANCE, true);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySearchEval.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySearchEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySearchEval.java
new file mode 100755
index 0000000..e3490db
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySearchEval.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+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.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractArraySearchEval implements IScalarEvaluator {
+    private final IPointable listArg;
+    private final IPointable searchedValueArg;
+    private final IScalarEvaluator listEval;
+    private final IScalarEvaluator searchedValueEval;
+    private final IBinaryComparator comp;
+    private final ListAccessor listAccessor;
+    private final SourceLocation sourceLocation;
+    protected final AMutableInt32 intValue;
+    protected final ArrayBackedValueStorage storage;
+
+    public AbstractArraySearchEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+            throws HyracksDataException {
+        storage = new ArrayBackedValueStorage();
+        listArg = new VoidPointable();
+        searchedValueArg = new VoidPointable();
+        listEval = args[0].createScalarEvaluator(ctx);
+        searchedValueEval = args[1].createScalarEvaluator(ctx);
+        comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        listAccessor = new ListAccessor();
+        intValue = new AMutableInt32(-1);
+        sourceLocation = sourceLoc;
+    }
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        // 1st arg: list
+        listEval.evaluate(tuple, listArg);
+        byte[] listBytes = listArg.getByteArray();
+        int listOffset = listArg.getStartOffset();
+
+        // 2nd arg: value to search for
+        searchedValueEval.evaluate(tuple, searchedValueArg);
+        byte[] valueBytes = searchedValueArg.getByteArray();
+        int valueOffset = searchedValueArg.getStartOffset();
+        int valueLength = searchedValueArg.getLength();
+
+        // for now, we don't support deep equality of object/lists. Throw an error if the value is of these types
+        if (ATYPETAGDESERIALIZER.deserialize(valueBytes[valueOffset]).isDerivedType()) {
+            throw HyracksDataException.create(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLocation);
+        }
+
+        if (!ATYPETAGDESERIALIZER.deserialize(listBytes[listOffset]).isListType()) {
+            PointableHelper.setNull(result);
+            return;
+        }
+
+        // initialize variables; -1 = value not found
+        intValue.setValue(-1);
+        listAccessor.reset(listBytes, listOffset);
+        int numItems = listAccessor.size();
+
+        try {
+            for (int i = 0; i < numItems; i++) {
+                storage.reset();
+                listAccessor.writeItem(i, storage.getDataOutput());
+                if (comp.compare(storage.getByteArray(), storage.getStartOffset(), storage.getLength(), valueBytes,
+                        valueOffset, valueLength) == 0) {
+                    intValue.setValue(i);
+                    break;
+                }
+            }
+            processResult(intValue, result);
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+    }
+
+    protected abstract void processResult(AMutableInt32 intValue, IPointable result) throws HyracksDataException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayContainsDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayContainsDescriptor.java
new file mode 100755
index 0000000..cc651f6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayContainsDescriptor.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public class ArrayContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArrayContainsDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_CONTAINS;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+                return new ArrayContainsFunction(args, ctx);
+            }
+        };
+    }
+
+    public class ArrayContainsFunction extends AbstractArraySearchEval {
+        private final ISerializerDeserializer booleanSerde;
+
+        public ArrayContainsFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            super(args, ctx, sourceLoc);
+            // TODO(ali): should we get the nontagged serde?
+            booleanSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+        }
+
+        @Override
+        public void processResult(AMutableInt32 intValue, IPointable result) throws HyracksDataException {
+            storage.reset();
+            booleanSerde.serialize(ABoolean.valueOf(intValue.getIntegerValue() != -1), storage.getDataOutput());
+            result.set(storage);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java
index caf93b0..5258abd 100755
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArrayPositionDescriptor.java
@@ -18,11 +18,6 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
-
-import java.io.IOException;
-
-import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.AMutableInt32;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -30,20 +25,14 @@ import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.ListAccessor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 public class ArrayPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
@@ -73,71 +62,17 @@ public class ArrayPositionDescriptor extends AbstractScalarFunctionDynamicDescri
         };
     }
 
-    public class ArrayPositionFunction implements IScalarEvaluator {
-        private final ArrayBackedValueStorage storage;
-        private final IPointable listArg;
-        private final IPointable searchedValueArg;
-        private final IScalarEvaluator listEval;
-        private final IScalarEvaluator searchedValueEval;
-        private final IBinaryComparator comp;
-        private final ListAccessor listAccessor;
-        private final AMutableInt32 intValue;
+    public class ArrayPositionFunction extends AbstractArraySearchEval {
         private final ISerializerDeserializer intSerde;
 
         public ArrayPositionFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
                 throws HyracksDataException {
-            storage = new ArrayBackedValueStorage();
-            listArg = new VoidPointable();
-            searchedValueArg = new VoidPointable();
-            listEval = args[0].createScalarEvaluator(ctx);
-            searchedValueEval = args[1].createScalarEvaluator(ctx);
-            comp = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-            listAccessor = new ListAccessor();
-            intValue = new AMutableInt32(-1);
+            super(args, ctx, sourceLoc);
             intSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
         }
 
         @Override
-        public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
-            // 1st arg: list
-            listEval.evaluate(tuple, listArg);
-            byte[] listBytes = listArg.getByteArray();
-            int listOffset = listArg.getStartOffset();
-
-            // 2nd arg: value to search for
-            searchedValueEval.evaluate(tuple, searchedValueArg);
-            byte[] valueBytes = searchedValueArg.getByteArray();
-            int valueOffset = searchedValueArg.getStartOffset();
-            int valueLength = searchedValueArg.getLength();
-
-            // for now, we don't support deep equality of object/lists. Throw an error if the value is of these types
-            if (ATYPETAGDESERIALIZER.deserialize(valueBytes[valueOffset]).isDerivedType()) {
-                throw HyracksDataException.create(ErrorCode.CANNOT_COMPARE_COMPLEX, sourceLoc);
-            }
-
-            if (!ATYPETAGDESERIALIZER.deserialize(listBytes[listOffset]).isListType()) {
-                PointableHelper.setNull(result);
-                return;
-            }
-
-            listAccessor.reset(listBytes, listOffset);
-            int numItems = listAccessor.size();
-            intValue.setValue(-1);
-
-            try {
-                for (int i = 0; i < numItems; i++) {
-                    storage.reset();
-                    listAccessor.writeItem(i, storage.getDataOutput());
-                    if (comp.compare(storage.getByteArray(), storage.getStartOffset(), storage.getLength(), valueBytes,
-                            valueOffset, valueLength) == 0) {
-                        intValue.setValue(i);
-                        break;
-                    }
-                }
-            } catch (IOException e) {
-                throw HyracksDataException.create(e);
-            }
-
+        public void processResult(AMutableInt32 intValue, IPointable result) throws HyracksDataException {
             storage.reset();
             intSerde.serialize(intValue, storage.getDataOutput());
             result.set(storage);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/9c9ed805/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 45dc76c..f89e67b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -144,6 +144,7 @@ import org.apache.asterix.runtime.evaluators.constructors.UnorderedListConstruct
 import org.apache.asterix.runtime.evaluators.functions.AndDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayAppendDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArrayContainsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayPositionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayReverseDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.CastTypeDescriptor;
@@ -376,6 +377,7 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.addGenerated(ArrayAppendDescriptor.FACTORY);
         fc.addGenerated(ArrayPositionDescriptor.FACTORY);
         fc.addGenerated(ArrayReverseDescriptor.FACTORY);
+        fc.addGenerated(ArrayContainsDescriptor.FACTORY);
 
         // unnesting functions
         fc.add(TidRunningAggregateDescriptor.FACTORY);