You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2018/11/27 18:55:39 UTC

[1/2] asterixdb git commit: [ASTERIXDB-2476][FUN] Support array slicing in SQL++

Repository: asterixdb
Updated Branches:
  refs/heads/master ae2d45498 -> 7090e74bb


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithoutEndPositionDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithoutEndPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithoutEndPositionDescriptor.java
new file mode 100644
index 0000000..2e6fb99
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithoutEndPositionDescriptor.java
@@ -0,0 +1,94 @@
+/*
+ * 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.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.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.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * array_slice(array, start) This function takes 2 arguments, {@code array} and {@code start} and returns
+ * a subset of the array that starts at position {@code start} to position {@code length - 1}.
+ */
+public class ArraySliceWithoutEndPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArraySliceWithoutEndPositionDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            // the type of the input list is needed in order to use the same type for the new returned list
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_SLICE_WITHOUT_END_POSITION;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @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 ArraySliceEval(args, ctx, sourceLoc);
+            }
+        };
+    }
+
+    public class ArraySliceEval extends AbstractArraySliceEval {
+
+        // Constructor
+        public ArraySliceEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+                throws HyracksDataException {
+            super(args, ctx, sourceLoc, ArraySliceWithoutEndPositionDescriptor.this.getIdentifier(), inputListType);
+        }
+
+        // Return null, no end position
+        @Override
+        protected IScalarEvaluator getEndPositionEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/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 dd1d862..2ba4b15 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
@@ -241,7 +241,9 @@ import org.apache.asterix.runtime.evaluators.functions.ArrayRemoveDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayRepeatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayReplaceDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayReverseDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArraySliceWithEndPositionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArraySortDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ArraySliceWithoutEndPositionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArrayStarDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ArraySymDiffnDescriptor;
@@ -498,6 +500,8 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.addGenerated(ArrayRangeDescriptor.FACTORY);
         fc.addGenerated(ArrayFlattenDescriptor.FACTORY);
         fc.add(ArrayReplaceDescriptor.FACTORY);
+        fc.addGenerated(ArraySliceWithEndPositionDescriptor.FACTORY);
+        fc.addGenerated(ArraySliceWithoutEndPositionDescriptor.FACTORY);
         fc.addGenerated(ArraySymDiffDescriptor.FACTORY);
         fc.addGenerated(ArraySymDiffnDescriptor.FACTORY);
         fc.addGenerated(ArrayStarDescriptor.FACTORY);


[2/2] asterixdb git commit: [ASTERIXDB-2476][FUN] Support array slicing in SQL++

Posted by dl...@apache.org.
[ASTERIXDB-2476][FUN] Support array slicing in SQL++

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

Details:
- Added array_slice function. 2 functions were added, one that
accepts 2 arguments and the other accepts 3 arguments. The
arguments are (list, start, end) respectively.
- Added array_slice function test cases.
- Array slice syntax and documentation will be added in subsequent
changes.

Change-Id: Ic609676cc068e92c6342fb0fa4c9074ec4bd98bf
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3031
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>


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

Branch: refs/heads/master
Commit: 7090e74bb6f1d03c983226d016e79afac0a1fa9d
Parents: ae2d454
Author: Hussain Towaileb <Hu...@Gmail.com>
Authored: Tue Nov 27 08:35:42 2018 +0300
Committer: Dmitry Lychagin <dm...@couchbase.com>
Committed: Tue Nov 27 10:55:21 2018 -0800

----------------------------------------------------------------------
 .../array_slice_double_argument.1.ddl.sqlpp     |  56 ++++
 .../array_slice_double_argument.2.update.sqlpp  |  27 ++
 .../array_slice_double_argument.3.query.sqlpp   |  27 ++
 .../array_slice_double_argument.4.ddl.sqlpp     |  20 ++
 .../array_slice_exception_result.1.ddl.sqlpp    |  56 ++++
 .../array_slice_exception_result.2.update.sqlpp |  27 ++
 .../array_slice_exception_result.3.query.sqlpp  |  26 ++
 .../array_slice_exception_result.4.ddl.sqlpp    |  20 ++
 .../array_slice_int_argument.1.ddl.sqlpp        |  56 ++++
 .../array_slice_int_argument.2.update.sqlpp     |  27 ++
 .../array_slice_int_argument.3.query.sqlpp      |  28 ++
 .../array_slice_int_argument.4.ddl.sqlpp        |  20 ++
 .../array_slice_missing_result.1.ddl.sqlpp      |  56 ++++
 .../array_slice_missing_result.2.update.sqlpp   |  27 ++
 .../array_slice_missing_result.3.query.sqlpp    |  30 ++
 .../array_slice_missing_result.4.ddl.sqlpp      |  20 ++
 .../array_slice_negative_argument.1.ddl.sqlpp   |  56 ++++
 ...array_slice_negative_argument.2.update.sqlpp |  27 ++
 .../array_slice_negative_argument.3.query.sqlpp |  27 ++
 .../array_slice_negative_argument.4.ddl.sqlpp   |  20 ++
 .../array_slice_null_result.1.ddl.sqlpp         |  56 ++++
 .../array_slice_null_result.2.update.sqlpp      |  27 ++
 .../array_slice_null_result.3.query.sqlpp       |  42 +++
 .../array_slice_null_result.4.ddl.sqlpp         |  20 ++
 .../array_slice_double_argument.3.adm           |   1 +
 .../array_slice_int_argument.3.adm              |   1 +
 .../array_slice_missing_result.3.adm            |   1 +
 .../array_slice_negative_argument.3.adm         |   1 +
 .../array_slice_null_result.3.adm               |   1 +
 .../resources/runtimets/testsuite_sqlpp.xml     |  31 ++
 .../asterix/om/functions/BuiltinFunctions.java  |   6 +
 .../om/typecomputer/impl/AListTypeComputer.java |  13 +-
 .../om/typecomputer/TypeComputerTest.java       |   2 +
 .../functions/AbstractArraySliceEval.java       | 315 +++++++++++++++++++
 .../functions/AbstractScalarEval.java           |  34 ++
 .../ArraySliceWithEndPositionDescriptor.java    |  95 ++++++
 .../ArraySliceWithoutEndPositionDescriptor.java |  94 ++++++
 .../runtime/functions/FunctionCollection.java   |   4 +
 38 files changed, 1396 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.query.sqlpp
new file mode 100755
index 0000000..00d0b2e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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": (array_slice([0, 1, 2, 3], 0, 1.0)),      // OK
+  "t2": (array_slice([0, 1, 2, 3], 2.0, 3.0)),    // OK
+  "t3": (array_slice([0, 1, 2, 3], 2, 3.0)),      // OK
+  "t4": (SELECT value array_slice(d1.followers, 0.0, 1.0) FROM d1) // OK
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.4.ddl.sqlpp
new file mode 100755
index 0000000..873178c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.4.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/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.3.query.sqlpp
new file mode 100755
index 0000000..82e0f00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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": (array_slice()),                       // Less than 2 arguments
+  "t2": (array_slice([0, 1, 2, 3])),           // Less than 2 arguments
+  "t3": (array_slice([0, 1, 2, 3], 0, 2, 4))   // More than 3 arguments
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.4.ddl.sqlpp
new file mode 100755
index 0000000..873178c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_exception_result/array_slice_exception_result.4.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/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.query.sqlpp
new file mode 100755
index 0000000..623d407
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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": (array_slice([0, 1, 2, 3], 0, 1)),    // OK
+  "t2": (array_slice([0, 1, 2, 3], 2, 4)),    // OK
+  "t3": (array_slice([0, 1, 2, 3], 0)),       // OK
+  "t4": (SELECT value array_slice(d1.followers, 0, 1) FROM d1), // OK
+  "t5": (SELECT value array_slice(d1.followers, 0) FROM d1) // OK
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.4.ddl.sqlpp
new file mode 100755
index 0000000..873178c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.4.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/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.query.sqlpp
new file mode 100755
index 0000000..71521de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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": (array_slice(missing, 0) IS MISSING),                // array is missing
+  "t2": (array_slice(missing, 0, 2) IS MISSING),             // array is missing
+  "t3": (array_slice([0, 1, 2, 3], missing) IS MISSING),     // start is missing
+  "t4": (array_slice([0, 1, 2, 3], missing, 2) IS MISSING),  // start is missing
+  "t5": (array_slice([0, 1, 2, 3], 0, missing) IS MISSING),  // end is missing
+  "t6": (SELECT value array_slice(d1.followers, missing) IS MISSING FROM d1),    // start is missing
+  "t7": (SELECT value array_slice(d1.followers, 0, 1) IS MISSING FROM d1)        // first record is missing (result is [ true, false ])
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.4.ddl.sqlpp
new file mode 100755
index 0000000..873178c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.4.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/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.query.sqlpp
new file mode 100755
index 0000000..41f33ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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": (array_slice([0, 1, 2, 3], 0, -1)),    // OK
+  "t2": (array_slice([0, 1, 2, 3], -2, 3)),    // OK
+  "t3": (array_slice([0, 1, 2, 3], -2, -1)),   // OK
+  "t4": (SELECT value array_slice(d1.followers, 0, -1) FROM d1) // OK
+};
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.4.ddl.sqlpp
new file mode 100755
index 0000000..873178c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.4.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/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.1.ddl.sqlpp
new file mode 100755
index 0000000..1c55a9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 type t1 AS {
+
+};
+
+create type t2 AS {
+id: int,
+compType: t1
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid hints (`CARDINALITY`=`100`);
+create dataset d1(t2) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.2.update.sqlpp
new file mode 100755
index 0000000..1c4bad0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.2.update.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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`));
+
+insert into d1([
+{"id":1, "compType":{"sth":33}},
+{"id":2, "compType":{"sth":44}, "followers":["John Green", "Emily Jones"]}
+]);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.query.sqlpp
new file mode 100755
index 0000000..4a20935
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.query.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * 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": (array_slice(null, 1) IS NULL),                 // array is null
+  "t2": (array_slice(null, 1, 2) IS NULL),              // array is null
+  "t3": (array_slice([0, 1, 2, 3], null) IS NULL),      // start is null
+  "t4": (array_slice([0, 1, 2, 3], null, 3) IS NULL),   // start is null
+  "t5": (array_slice([0, 1, 2, 3], 0, null) IS NULL),   // end is null
+  "t6": (array_slice([0, 1, 2, 3], 1.5, 3) IS NULL),    // invalid start
+  "t7": (array_slice([0, 1, 2, 3], 1, -2.3) IS NULL),   // invalid end
+  "t8": (array_slice([0, 1, 2, 3], 2, 1) IS NULL),      // start > end
+  "t9": (array_slice([0, 1, 2, 3], 10, 3) IS NULL),     // start > length
+  "t10": (array_slice([0, 1, 2, 3], 1, 10) IS NULL),     // end > length
+  "t11": (array_slice([0, 1, 2, 3], 4, 4) IS NULL),      // start = length
+  "t12": (array_slice("non_array", 1, 2) IS NULL),      // Not an array
+  "t13": (array_slice([0, 1, 2, 3], "0", 3) IS NULL),   // Invalid start
+  "t14": (array_slice([0, 1, 2, 3], 0, "3") IS NULL),   // Invalid end
+  "t15": (SELECT value array_slice(d1.followers, null) IS NULL FROM d1),    // start is null
+  "t16": (SELECT value array_slice(d1.followers, null, 2) IS NULL FROM d1)  // start is null
+};
+
+// t15 has MISSING value in first record, so result will actually be [ null, true ]
+// t16 has MISSING value in first record, so result will actually be [ null, true ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.4.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.4.ddl.sqlpp
new file mode 100755
index 0000000..873178c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_slice/array_slice_null_result/array_slice_null_result.4.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/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.adm
new file mode 100644
index 0000000..01a22a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_double_argument/array_slice_double_argument.3.adm
@@ -0,0 +1 @@
+{ "t1": [ 0 ], "t2": [ 2 ], "t3": [ 2 ], "t4": [ null, [ "John Green" ] ] }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.adm
new file mode 100644
index 0000000..3b96f60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_int_argument/array_slice_int_argument.3.adm
@@ -0,0 +1 @@
+{ "t1": [ 0 ], "t2": [ 2, 3 ], "t3": [ 0, 1, 2, 3 ], "t4": [ null, [ "John Green" ] ], "t5": [ null, [ "John Green", "Emily Jones" ] ] }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.adm
new file mode 100644
index 0000000..7bec821
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_missing_result/array_slice_missing_result.3.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": [ true, true ], "t7": [ true, false ] }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.adm
new file mode 100644
index 0000000..23328e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_negative_argument/array_slice_negative_argument.3.adm
@@ -0,0 +1 @@
+{ "t1": [ 0, 1, 2 ], "t2": [ 2 ], "t3": [ 2 ], "t4": [ null, [ "John Green" ] ] }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.adm
new file mode 100644
index 0000000..1186784
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_slice/array_slice_null_result/array_slice_null_result.3.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true, "t9": true, "t10": true, "t11": true, "t12": true, "t13": true, "t14": true, "t15": [ null, true ], "t16": [ null, true ] }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/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 56dbfa1..362f722 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -2041,6 +2041,37 @@
         <expected-error>Cannot compare non-primitive values (in line 27, at column 8)</expected-error>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_double_argument">
+        <output-dir compare="Text">array_slice/array_slice_double_argument</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_int_argument">
+        <output-dir compare="Text">array_slice/array_slice_int_argument</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_negative_argument">
+        <output-dir compare="Text">array_slice/array_slice_negative_argument</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_missing_result">
+        <output-dir compare="Text">array_slice/array_slice_missing_result</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_null_result">
+        <output-dir compare="Text">array_slice/array_slice_null_result</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="array_fun">
+      <compilation-unit name="array_slice/array_slice_exception_result">
+        <output-dir compare="Text">array_slice/array_slice_exception_result</output-dir>
+        <expected-error>Compilation error</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/7090e74b/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 9a52f55..496321a 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
@@ -239,6 +239,10 @@ public class BuiltinFunctions {
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-symdiffn", FunctionIdentifier.VARARGS);
     public static final FunctionIdentifier ARRAY_STAR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-star", 1);
+    public static final FunctionIdentifier ARRAY_SLICE_WITHOUT_END_POSITION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-slice", 2);
+    public static final FunctionIdentifier ARRAY_SLICE_WITH_END_POSITION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "array-slice", 3);
 
     // objects
     public static final FunctionIdentifier RECORD_MERGE =
@@ -1923,6 +1927,8 @@ public class BuiltinFunctions {
         addFunction(ARRAY_SYMDIFF, AListMultiListArgsTypeComputer.INSTANCE, true);
         addFunction(ARRAY_SYMDIFFN, AListMultiListArgsTypeComputer.INSTANCE, true);
         addFunction(ARRAY_STAR, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(ARRAY_SLICE_WITH_END_POSITION, AListTypeComputer.INSTANCE_SLICE, true);
+        addFunction(ARRAY_SLICE_WITHOUT_END_POSITION, AListTypeComputer.INSTANCE_SLICE, true);
 
         // objects
         addFunction(RECORD_MERGE, RecordMergeTypeComputer.INSTANCE, true);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
index d1c9c48..8d3824a 100755
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AListTypeComputer.java
@@ -40,25 +40,36 @@ public class AListTypeComputer extends AbstractResultTypeComputer {
     public static final AListTypeComputer INSTANCE_APPEND = new AListTypeComputer(2, -1, false, true, false);
     public static final AListTypeComputer INSTANCE_INSERT = new AListTypeComputer(3, -1, false, true, false);
     public static final AListTypeComputer INSTANCE_REPLACE = new AListTypeComputer(3, 4, false, true, false);
+    public static final AListTypeComputer INSTANCE_SLICE = new AListTypeComputer(-1, -1, false, false, true, false);
 
     private final int minNumArgs;
     private final int maxNumArgs;
     private final boolean listIsLast;
     private final boolean makeOpen;
     private final boolean nullInNullOut;
+    private final boolean isCheckArgumentsCount;
 
+    // Maintain the old constructor
     private AListTypeComputer(int minNumArgs, int maxNumArgs, boolean listIsLast, boolean makeOpen,
             boolean nullInNullOut) {
+        this(minNumArgs, maxNumArgs, listIsLast, makeOpen, nullInNullOut, true);
+    }
+
+    // Use this constructor to skip checking the arguments count
+    private AListTypeComputer(int minNumArgs, int maxNumArgs, boolean listIsLast, boolean makeOpen,
+            boolean nullInNullOut, boolean isCheckArgumentsCount) {
         this.minNumArgs = minNumArgs;
         this.maxNumArgs = maxNumArgs;
         this.listIsLast = listIsLast;
         this.makeOpen = makeOpen;
         this.nullInNullOut = nullInNullOut;
+        this.isCheckArgumentsCount = isCheckArgumentsCount;
     }
 
     @Override
     protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
-        if (strippedInputTypes.length < minNumArgs || (maxNumArgs > 0 && strippedInputTypes.length > maxNumArgs)) {
+        if (isCheckArgumentsCount && (strippedInputTypes.length < minNumArgs
+                || (maxNumArgs > 0 && strippedInputTypes.length > maxNumArgs))) {
             String functionName = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier().getName();
             throw new CompilationException(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, expr.getSourceLocation(),
                     functionName);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java
index d18472c..4733c2f 100644
--- a/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/om/typecomputer/TypeComputerTest.java
@@ -78,9 +78,11 @@ public class TypeComputerTest {
 
         // functions that check the number of args inside the type computer
         List<Mutable<ILogicalExpression>> replaceArgs = createArgs(4, mockTypeEnv);
+        List<Mutable<ILogicalExpression>> sliceArgs = createArgs(3, mockTypeEnv);
         List<Mutable<ILogicalExpression>> rangeArgs = createArgs(3, mockTypeEnv);
         HashMap<String, List<Mutable<ILogicalExpression>>> map = new HashMap<>();
         map.put("INSTANCE_REPLACE", replaceArgs);
+        map.put("INSTANCE_SLICE", sliceArgs);
         map.put("ArrayRangeTypeComputer", rangeArgs);
 
         // Several exceptional type computers.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySliceEval.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySliceEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySliceEval.java
new file mode 100644
index 0000000..571e4d2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractArraySliceEval.java
@@ -0,0 +1,315 @@
+/*
+ * 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.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.UnorderedListBuilder;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+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;
+
+import java.io.IOException;
+
+import static org.apache.asterix.om.types.EnumDeserializer.ATYPETAGDESERIALIZER;
+
+/**
+ * <pre>
+ * array_slice(array, start, end) returns a subset of the {@code array} containing the elements from position
+ * {@code start} to position {@code end -1}. The {@code end} argument is optional. if {@code end} is not provided,
+ * the returned subset of the {@code array} contains the elements from {@code start} position to the end of
+ * {@code array}. The array index starts at {@code 0}. The element at {@code start} is included while the element
+ * at {@code end} is not included. Negative positions are counted backwards from the end of the array.
+ *
+ * Examples:
+ * array_slice([1, 2, 3, 4], 1, 3) will return [2, 3].
+ * array_slice([1, 2, 3, 4], -3, 3) will return [2, 3].
+ * array_slice([1, 2, 3, 4], 1) will return [2, 3, 4].
+ *
+ * It throws an error at compile time if the number of arguments is greater than {@code 3} or less than {@code 2}.
+ *
+ * {@code NULL} is returned if:
+ *  - {@code array} is not an array.
+ *  - {@code start} or {@code end} is not valid numbers. 1, 2, 3.0 are accepted but 3.2 is not accepted.
+ *  - {@code start} or {@code end} is greater than the length of the {@code array}.
+ *  - {@code end} is smaller than {@code start}.
+ * </pre>
+ */
+
+public abstract class AbstractArraySliceEval extends AbstractScalarEval {
+    // List type
+    private final IAType inputListType;
+
+    // Storage
+    private final ArrayBackedValueStorage storage = new ArrayBackedValueStorage();
+
+    // Evaluators
+    private final IScalarEvaluator listEval;
+    private final IScalarEvaluator startPositionEval;
+    private final IScalarEvaluator endPositionEval;
+
+    // Pointables
+    private final IPointable listPointable = new VoidPointable();
+    private final IPointable startPositionPointable = new VoidPointable();
+    private final IPointable endPositionPointable = new VoidPointable();
+
+    // Accessors
+    private final ListAccessor listAccessor = new ListAccessor();
+
+    // List Builders
+    private IAsterixListBuilder orderedListBuilder = new OrderedListBuilder();
+    private IAsterixListBuilder unorderedListBuilder = new UnorderedListBuilder();
+
+    // Constructor
+    public AbstractArraySliceEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc,
+            FunctionIdentifier functionIdentifier, IAType inputListType) throws HyracksDataException {
+        // Source location
+        super(sourceLoc, functionIdentifier);
+
+        // List type
+        this.inputListType = inputListType;
+
+        // Evaluators
+        listEval = args[0].createScalarEvaluator(ctx);
+        startPositionEval = args[1].createScalarEvaluator(ctx);
+
+        // Check for optional parameter
+        endPositionEval = getEndPositionEval(args, ctx);
+    }
+
+    protected abstract IScalarEvaluator getEndPositionEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+            throws HyracksDataException;
+
+    @Override
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+        // Reset storage
+        storage.reset();
+
+        // Evaluate
+        listEval.evaluate(tuple, listPointable);
+        startPositionEval.evaluate(tuple, startPositionPointable);
+        if (endPositionEval != null) {
+            endPositionEval.evaluate(tuple, endPositionPointable);
+        }
+
+        // Positions
+        int startPositionValue;
+        int endPositionValue = 0;
+
+        // Data bytes, offsets and type tags
+        byte[] listBytes = listPointable.getByteArray();
+        int listOffset = listPointable.getStartOffset();
+        ATypeTag listTypetag = ATYPETAGDESERIALIZER.deserialize(listBytes[listOffset]);
+        byte[] startPositionBytes = startPositionPointable.getByteArray();
+        int startPositionOffset = startPositionPointable.getStartOffset();
+        ATypeTag startPositionTypeTag = ATYPETAGDESERIALIZER.deserialize(startPositionBytes[startPositionOffset]);
+
+        // Invalid types checks
+        if (!listTypetag.isListType() || !ATypeHierarchy.isCompatible(startPositionTypeTag, ATypeTag.DOUBLE)) {
+            PointableHelper.setNull(result);
+            return;
+        }
+
+        // List accessor
+        listAccessor.reset(listBytes, listOffset);
+
+        // Optional parameter
+        byte[] endPositionBytes = null;
+        int endPositionOffset = 0;
+        ATypeTag endPositionTypeTag;
+
+        // End position available or use list length as end position
+        if (endPositionEval != null) {
+            endPositionBytes = endPositionPointable.getByteArray();
+            endPositionOffset = endPositionPointable.getStartOffset();
+            endPositionTypeTag = ATYPETAGDESERIALIZER.deserialize(endPositionBytes[endPositionOffset]);
+
+            // Invalid types checks
+            if (!ATypeHierarchy.isCompatible(endPositionTypeTag, ATypeTag.DOUBLE)) {
+                PointableHelper.setNull(result);
+                return;
+            }
+        } else {
+            // Use list length as end position
+            endPositionValue = listAccessor.size();
+            endPositionTypeTag = ATypeTag.BIGINT;
+        }
+
+        // From here onward, all arguments are available and compatible
+        // Get the position value
+        try {
+            startPositionValue = getValue(startPositionBytes, startPositionOffset + 1, startPositionTypeTag);
+            endPositionValue = endPositionEval != null
+                    ? getValue(endPositionBytes, endPositionOffset + 1, endPositionTypeTag) : endPositionValue;
+        } catch (HyracksDataException ignored) { // NOSONAR: Ignore the exception, invalid number returns null
+            PointableHelper.setNull(result);
+            return;
+        }
+
+        // Since we accept negative values for positions, we need to convert them appropriately to positives before
+        // we compare the start and end positions. (e.g. length = 4, start = -1 -> start = 4 + (-1) = 3)
+        startPositionValue = startPositionValue < 0 ? listAccessor.size() + startPositionValue : startPositionValue;
+        endPositionValue = endPositionValue < 0 ? listAccessor.size() + endPositionValue : endPositionValue;
+
+        // Check arguments validity
+        if (!isValidArguments(listAccessor.size(), startPositionValue, endPositionValue)) {
+            PointableHelper.setNull(result);
+            return;
+        }
+
+        // From here onward, all arguments are valid
+        // List builder & collection type
+        AbstractCollectionType collectionType;
+        IAsterixListBuilder listBuilder =
+                listAccessor.getListType() == ATypeTag.ARRAY ? orderedListBuilder : unorderedListBuilder;
+
+        // Unknown list type
+        if (!inputListType.getTypeTag().isListType()) {
+            // Get the list item type using the type tag
+            ATypeTag listItemTypeTag = listAccessor.getItemType();
+            IAType listItemType = TypeTagUtil.getBuiltinTypeByTag(listItemTypeTag);
+
+            // List of type listItemType
+            if (listAccessor.getListType() == ATypeTag.ARRAY) {
+                collectionType = new AOrderedListType(listItemType, listItemType.getTypeName());
+            } else {
+                collectionType = new AUnorderedListType(listItemType, listItemType.getTypeName());
+            }
+        }
+        // Known list type, use it directly
+        else {
+            collectionType = (AbstractCollectionType) inputListType;
+        }
+
+        // Builder list type
+        listBuilder.reset(collectionType);
+
+        try {
+            // Create the subset list based on the positions
+            for (int i = startPositionValue; i < endPositionValue; i++) {
+                storage.reset();
+                listAccessor.writeItem(i, storage.getDataOutput());
+                listBuilder.addItem(storage);
+            }
+        } catch (IOException ex) {
+            throw HyracksDataException.create(ex);
+        }
+
+        // Final result
+        storage.reset();
+        listBuilder.write(storage.getDataOutput(), true);
+        result.set(storage);
+    }
+
+    // Get the value
+    private int getValue(byte[] data, int offset, ATypeTag typeTag) throws HyracksDataException {
+        // Using double since we accept values like 3.0, but not 3.5
+        double value;
+
+        // Value based on type tag
+        switch (typeTag) {
+            case TINYINT:
+                value = AInt8SerializerDeserializer.getByte(data, offset);
+                break;
+            case SMALLINT:
+                value = AInt16SerializerDeserializer.getShort(data, offset);
+                break;
+            case INTEGER:
+                value = AInt32SerializerDeserializer.getInt(data, offset);
+                break;
+            case BIGINT:
+                value = AInt64SerializerDeserializer.getLong(data, offset);
+                break;
+            case FLOAT:
+                value = AFloatSerializerDeserializer.getFloat(data, offset);
+                break;
+            case DOUBLE:
+                value = ADoubleSerializerDeserializer.getDouble(data, offset);
+                break;
+            default:
+                throw new UnsupportedItemTypeException(sourceLoc, functionIdentifier, typeTag.serialize());
+        }
+
+        // Values like 1, 2, 3.0 are ok, but 0.3 and 3.5 are not accepted
+        if (value > Math.floor(value)) {
+            throw new InvalidDataFormatException(sourceLoc, functionIdentifier, typeTag.serialize());
+        }
+
+        return (int) value;
+    }
+
+    /**
+     * Ensures that the positions used are valid
+     *
+     * @param listLength list length
+     * @param startPosition start position value
+     * @param endPosition end position value
+     *
+     * @return {@code true} if all conditions are valid, otherwise {@code false}
+     */
+    private boolean isValidArguments(double listLength, double startPosition, double endPosition) {
+
+        // Negative values check (negative positions already adjusted, if a value is still negative then it's
+        // greater than the list length)
+        if (startPosition < 0 || endPosition < 0) {
+            return false;
+        }
+
+        // Length vs Position check
+        if (startPosition > listLength - 1 || endPosition > listLength) {
+            return false;
+        }
+
+        // Value validity check (1, 2, 3.0 are accepted, but 3.2 is not)
+        if (startPosition > Math.floor(startPosition) || endPosition > Math.floor(endPosition)) {
+            return false;
+        }
+
+        // Start vs end position check (start position can't be greater than end position)
+        if (startPosition > endPosition) {
+            return false;
+        }
+
+        // All conditions passed
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractScalarEval.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractScalarEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractScalarEval.java
new file mode 100644
index 0000000..6cb3666
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractScalarEval.java
@@ -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.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AbstractScalarEval implements IScalarEvaluator {
+    protected final SourceLocation sourceLoc;
+    protected final FunctionIdentifier functionIdentifier;
+
+    public AbstractScalarEval(SourceLocation sourceLoc, FunctionIdentifier functionIdentifier) {
+        this.sourceLoc = sourceLoc;
+        this.functionIdentifier = functionIdentifier;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/7090e74b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithEndPositionDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithEndPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithEndPositionDescriptor.java
new file mode 100644
index 0000000..da0053a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ArraySliceWithEndPositionDescriptor.java
@@ -0,0 +1,95 @@
+/*
+ * 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.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.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.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * array_slice(array, start, end) This function takes 3 arguments, {@code array}, {@code start} and {@code end} and
+ * returns a subset of the array that starts at position {@code start} to position {@code end - 1}
+ */
+public class ArraySliceWithEndPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private IAType inputListType;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ArraySliceWithEndPositionDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            // the type of the input list is needed in order to use the same type for the new returned list
+            return FunctionTypeInferers.SET_ARGUMENT_TYPE;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ARRAY_SLICE_WITH_END_POSITION;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        inputListType = (IAType) states[0];
+    }
+
+    @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 ArraySliceEval(args, ctx, sourceLoc);
+            }
+        };
+    }
+
+    public class ArraySliceEval extends AbstractArraySliceEval {
+
+        // Constructor
+        public ArraySliceEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, SourceLocation sourceLoc)
+                throws HyracksDataException {
+            super(args, ctx, sourceLoc, ArraySliceWithEndPositionDescriptor.this.getIdentifier(), inputListType);
+        }
+
+        // Return end position evaluator
+        @Override
+        protected IScalarEvaluator getEndPositionEval(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            return args[2].createScalarEvaluator(ctx);
+        }
+    }
+}