You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by ji...@apache.org on 2015/10/29 05:45:04 UTC

[13/15] incubator-asterixdb git commit: ASTERIXDB-1102: VarSize Encoding to store length of String and ByteArray

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java b/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java
index b9b5527..1b7bd1e 100644
--- a/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java
+++ b/asterix-app/src/main/java/org/apache/asterix/file/ExternalIndexingOperations.java
@@ -235,9 +235,10 @@ public class ExternalIndexingOperations {
                 .splitProviderAndPartitionConstraintsForFilesIndex(dataset.getDataverseName(),
                         dataset.getDatasetName(), getFilesIndexName(dataset.getDatasetName()), true);
         IFileSplitProvider secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+        FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
         ILocalResourceMetadata localResourceMetadata = new ExternalBTreeLocalResourceMetadata(
-                FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
-                FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, false, dataset.getDatasetId(),
+                filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+                filesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, false, dataset.getDatasetId(),
                 mergePolicyFactory, mergePolicyFactoryProperties);
         PersistentLocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.ExternalBTreeResource);
@@ -263,7 +264,6 @@ public class ExternalIndexingOperations {
      * @param jobSpec
      * @param itemType
      * @param dataset
-     * @param format
      * @param files
      * @param indexerDesc
      * @return
@@ -751,10 +751,11 @@ public class ExternalIndexingOperations {
                         dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties.getBloomFilterFalsePositiveRate(),
                 ExternalDatasetsRegistry.INSTANCE.getDatasetVersion(dataset), true);
+        FilesIndexDescription filesIndexDescription = new FilesIndexDescription();
         LSMTreeIndexCompactOperatorDescriptor compactOp = new LSMTreeIndexCompactOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                secondaryFileSplitProvider, FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
-                FilesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, indexDataflowHelperFactory,
+                secondaryFileSplitProvider, filesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS,
+                filesIndexDescription.FILES_INDEX_COMP_FACTORIES, new int[] { 0 }, indexDataflowHelperFactory,
                 NoOpOperationCallbackFactory.INSTANCE);
         spec.addRoot(compactOp);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, compactOp,

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
index 579a41f..9524e0d 100644
--- a/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
@@ -19,20 +19,12 @@
 package org.apache.asterix.test.runtime;
 
 import java.io.File;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.common.config.AsterixPropertiesAccessor;
 import org.apache.asterix.common.config.AsterixTransactionProperties;
@@ -41,6 +33,16 @@ import org.apache.asterix.external.dataset.adapter.FileSystemBasedAdapter;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.test.aql.TestsUtils;
 import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.xml.TestGroup;
+import org.apache.asterix.testframework.xml.TestSuite;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  * Runs the runtime test cases under 'asterix-app/src/test/resources/runtimets'.
@@ -58,6 +60,8 @@ public class ExecutionTest {
 
     protected static AsterixTransactionProperties txnProperties;
 
+    protected static TestGroup FailedGroup;
+
     @BeforeClass
     public static void setUp() throws Exception {
         System.out.println("Starting setup");
@@ -88,6 +92,9 @@ public class ExecutionTest {
         // to be node controller ids; a valid assumption in test environment.
         System.setProperty(FileSystemBasedAdapter.NODE_RESOLVER_FACTORY_PROPERTY,
                 IdentitiyResolverFactory.class.getName());
+
+        FailedGroup = new TestGroup();
+        FailedGroup.setName("failed");
     }
 
     @AfterClass
@@ -103,6 +110,24 @@ public class ExecutionTest {
             TestsUtils.deleteRec(new File(d));
         }
         HDFSCluster.getInstance().cleanup();
+
+        if (FailedGroup != null && FailedGroup.getTestCase().size() > 0) {
+            File temp = File.createTempFile("failed", ".xml");
+            javax.xml.bind.JAXBContext jaxbCtx = null;
+            jaxbCtx = javax.xml.bind.JAXBContext.newInstance(TestSuite.class.getPackage().getName());
+            javax.xml.bind.Marshaller marshaller = null;
+            marshaller = jaxbCtx.createMarshaller();
+            marshaller.setProperty(javax.xml.bind.Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+            TestSuite failedSuite = new TestSuite();
+            failedSuite.setResultOffsetPath("results");
+            failedSuite.setQueryOffsetPath("queries");
+            failedSuite.getTestGroup().add(FailedGroup);
+            marshaller.marshal(failedSuite, temp);
+            System.err.println("The failed.xml is written to :" + temp.getAbsolutePath() +
+                    ". You can copy it to only.xml by the following cmd:" +
+                    "\rcp " + temp.getAbsolutePath() + " " +
+                    Paths.get("./src/test/resources/runtimets/only.xml").toAbsolutePath());
+        }
     }
 
     private static void deleteTransactionLogs() throws Exception {
@@ -141,6 +166,6 @@ public class ExecutionTest {
 
     @Test
     public void test() throws Exception {
-            TestsUtils.executeTest(PATH_ACTUAL, tcCtx, null, false);
+        TestsUtils.executeTest(PATH_ACTUAL, tcCtx, null, false, FailedGroup);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/only.xml
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/only.xml b/asterix-app/src/test/resources/runtimets/only.xml
index f1a0a2f..245401b 100644
--- a/asterix-app/src/test/resources/runtimets/only.xml
+++ b/asterix-app/src/test/resources/runtimets/only.xml
@@ -1,3 +1,4 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
 <!--
  ! Licensed to the Apache Software Foundation (ASF) under one
  ! or more contributor license agreements.  See the NOTICE file
@@ -16,12 +17,7 @@
  ! specific language governing permissions and limitations
  ! under the License.
  !-->
-<test-suite
-    xmlns="urn:xml.testframework.asterix.apache.org"
-    ResultOffsetPath="results"
-    QueryOffsetPath="queries"
-    QueryFileExtension=".aql">
-    <test-group name="debug">
-
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries">
+    <test-group name="failed">
     </test-group>
 </test-suite>

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql
index 9aaf591..8afb57e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_groupby-2/big_object_groupby.3.query.aql
@@ -27,7 +27,7 @@ use dataverse test;
 
 for $i in dataset('Line')
 group by $comment := $i.l_comment, $id := $i.l_orderkey with $i
-order by $id
+order by $id, string-length($comment), $comment
 return {
     "id": $id,
     "length": string-length($comment),

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.1.ddl.aql
new file mode 100644
index 0000000..14ae072
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.1.ddl.aql
@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineType as closed {
+  l_orderkey: int32,
+  l_partkey: int32,
+  l_suppkey: int32,
+  l_linenumber: int32,
+  l_quantity: double,
+  l_extendedprice: double,
+  l_discount: double,
+  l_tax: double,
+  l_returnflag: string,
+  l_linestatus: string,
+  l_shipdate: string,
+  l_commitdate: string,
+  l_receiptdate: string,
+  l_shipinstruct: string,
+  l_shipmode: string,
+  l_comment: string
+}
+
+create type OrderType as closed {
+  o_orderkey: int32,
+  o_custkey: int32,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int32,
+  o_comment: string
+}
+
+create type CustomerType as closed {
+  c_custkey: int32,
+  c_name: string,
+  c_address: string,
+  c_nationkey: int32,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+}
+
+
+create external dataset Line(LineType)
+using localfs
+(("path"="nc1://data/big-object/lineitem.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Order(OrderType)
+using localfs
+(("path"="nc1://data/big-object/order.tbl.verylong.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+create external dataset Customer(CustomerType)
+using localfs
+(("path"="nc1://data/big-object/customer.tbl.big"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.2.update.aql
new file mode 100644
index 0000000..538b48c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.3.query.aql
new file mode 100644
index 0000000..9ae41eb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_load/big_object_load.3.query.aql
@@ -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.
+ */
+/*
+* Description  : Create an external dataset that contains long comments fields, 10% of the records have a 32K size comments.
+                 This will trigger into the VSizeFrame path
+* Expected Res : Success
+* Date         : Jun 16 2015
+*/
+
+use dataverse test;
+
+for $o in dataset('Order')
+return {
+    "custkey": $o.o_custkey,
+    "orderkey": $o.o_orderkey,
+    "len-comment": string-length($o.o_comment)
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
index ff22348..58f6a96 100644
--- a/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
@@ -30,4 +30,4 @@ let $r7 := find-binary($x, hex('ccddaabb')) = 0
 let $r8 := find-binary($x, null)
 let $r9 := find-binary(null, null)
 let $r0 := find-binary(null, $x)
-return [ $r1 and $r2 and $r3 and $r4 and $r5 and $r6 and $r7, $r8, $r9 , $r0]
+return [ $r1 , $r2 , $r3 , $r4 , $r5 , $r6 , $r7, $r8, $r9 , $r0]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
index f3046f9..d4dce94 100644
--- a/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
@@ -37,5 +37,5 @@ let $r12 := sub-binary($x, 1, 256) = $x
 let $r13 := sub-binary($x, 2, 256) = hex('bbccdd')
 let $r14 := sub-binary($x, 2, -1) = hex('')
 
-return  $r1 and $r2 and $r3 and $r4 and $r5 and $r6 and $r7 and $r8 and $r9 and $r10 and $r11 and $r12 and $r13 and $r14 
+return  [ $r1 ,$r2 ,$r3 ,$r4 ,$r5 , $r6 , $r7 , $r8 , $r9 , $r10 , $r11 , $r12 , $r13 , $r14 ]
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/boolean/and_01/and_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/boolean/and_01/and_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/boolean/and_01/and_01.3.query.aql
index 7fbaac5..eee9bf4 100644
--- a/asterix-app/src/test/resources/runtimets/queries/boolean/and_01/and_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/boolean/and_01/and_01.3.query.aql
@@ -18,6 +18,6 @@
  */
 use dataverse test;
 
-for $x in ["true"]
-for $y in ["false"]
+let $x := boolean("true")
+let $y := boolean("false")
 return $x and $y

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql
index 952f64b..1190479 100644
--- a/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/delete-from-loaded-dataset-with-index/delete-from-loaded-dataset-with-index.5.query.aql
@@ -21,4 +21,5 @@ use dataverse TinySocial;
 
 for $c in dataset('TweetMessages')
 where $c.user.statuses_count<473
+order by $c.tweetid
 return $c

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.5.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.5.query.aql
index 0d5e63b..45f38e9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.5.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/nested-index-dml/insert-into-loaded-dataset-with-index_02/insert-into-loaded-dataset-with-index_02.5.query.aql
@@ -21,5 +21,6 @@ use dataverse TinySocial;
 
 for $c in dataset('TweetMessages')
 where $c.user.name < "Nathan Giesen" and $c.user.statuses_count < 473
+order by $c.tweetid
 return $c
 

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.1.ddl.aql
deleted file mode 100644
index d17ea60..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.1.ddl.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 test if exists;
-create dataverse test;
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.2.update.aql
deleted file mode 100644
index 042f3ce..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.2.update.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.
- */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.3.query.aql
deleted file mode 100644
index e4d6370..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with1/end-with1.3.query.aql
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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 dataverse test;
-
-let $c1 := end-with("hello world","werld")
-return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.1.ddl.aql
deleted file mode 100644
index d17ea60..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.1.ddl.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 test if exists;
-create dataverse test;
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.2.update.aql
deleted file mode 100644
index 042f3ce..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.2.update.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.
- */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.3.query.aql
deleted file mode 100644
index 9250512..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with2/end-with2.3.query.aql
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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 dataverse test;
-
-let $c1 := end-with("hello world"," world")
-return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.1.ddl.aql
deleted file mode 100644
index d17ea60..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.1.ddl.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 test if exists;
-create dataverse test;
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.2.update.aql
deleted file mode 100644
index 042f3ce..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.2.update.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.
- */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.3.query.aql
deleted file mode 100644
index ca80dd0..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with3/end-with3.3.query.aql
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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 dataverse test;
-
-let $c1 := end-with("ends","")
-return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.1.ddl.aql
deleted file mode 100644
index d17ea60..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.1.ddl.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 test if exists;
-create dataverse test;
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.2.update.aql
deleted file mode 100644
index 042f3ce..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.2.update.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.
- */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.3.query.aql
deleted file mode 100644
index 4784297..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with4/end-with4.3.query.aql
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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 dataverse test;
-
-let $c1 := end-with("ends","ss")
-return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.1.ddl.aql
deleted file mode 100644
index d17ea60..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.1.ddl.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 test if exists;
-create dataverse test;
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.2.update.aql
deleted file mode 100644
index 042f3ce..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.2.update.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.
- */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.3.query.aql
deleted file mode 100644
index ff4d0df..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/end-with5/end-with5.3.query.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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 dataverse test;
-
-let $c1 := end-with("ends","s")
-let $c2 := end-with("start",null)
-let $c3 := end-with(null,null)
-let $c4 := end-with("",null)
-let $c5 := end-with("","")
-let $c6 := end-with(null,"")
-
-return {"f1": $c1, "f2": $c2, "f3": $c3, "f4": $c4, "f5": $c5, "f6": $c6}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.1.ddl.aql
new file mode 100644
index 0000000..d17ea60
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.1.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.2.update.aql
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.2.update.aql
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.3.query.aql
new file mode 100644
index 0000000..0dea842
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with1/ends-with1.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+let $c1 := ends-with("hello world","werld")
+return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.1.ddl.aql
new file mode 100644
index 0000000..d17ea60
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.1.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.2.update.aql
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.2.update.aql
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.3.query.aql
new file mode 100644
index 0000000..035d757
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with2/ends-with2.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+let $c1 := ends-with("hello world"," world")
+return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.1.ddl.aql
new file mode 100644
index 0000000..d17ea60
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.1.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.2.update.aql
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.2.update.aql
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.3.query.aql
new file mode 100644
index 0000000..477138b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with3/ends-with3.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+let $c1 := ends-with("ends","")
+return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.1.ddl.aql
new file mode 100644
index 0000000..d17ea60
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.1.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.2.update.aql
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.2.update.aql
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.3.query.aql
new file mode 100644
index 0000000..9bf1f5b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with4/ends-with4.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use dataverse test;
+
+let $c1 := ends-with("ends","ss")
+return {"result1": $c1}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.1.ddl.aql
new file mode 100644
index 0000000..d17ea60
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.1.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.2.update.aql
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.2.update.aql
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.3.query.aql
new file mode 100644
index 0000000..8a04dd8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with5/ends-with5.3.query.aql
@@ -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 dataverse test;
+
+let $c1 := ends-with("ends","s")
+let $c2 := ends-with("start",null)
+let $c3 := ends-with(null,null)
+let $c4 := ends-with("",null)
+let $c5 := ends-with("","")
+let $c6 := ends-with(null,"")
+
+return {"f1": $c1, "f2": $c2, "f3": $c3, "f4": $c4, "f5": $c5, "f6": $c6}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.1.ddl.aql
new file mode 100644
index 0000000..d17ea60
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.1.ddl.aql
@@ -0,0 +1,21 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.2.update.aql
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.2.update.aql
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.3.query.aql
new file mode 100644
index 0000000..9802279
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with6/ends-with6.3.query.aql
@@ -0,0 +1,23 @@
+/*
+ * 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 dataverse test;
+
+for $x in ["foofoo"]
+for $y in ["barbar"]
+return [ends-with($x, "ar"), ends-with($y, "ar")]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.1.ddl.aql
new file mode 100644
index 0000000..a8f38a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/*
+ * Testcase Name : endwith02.aql
+ * Description   : Positive tests
+ * Success       : Yes
+ * Date          : 20th April 2012
+ */
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.2.update.aql
new file mode 100644
index 0000000..a8f38a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/*
+ * Testcase Name : endwith02.aql
+ * Description   : Positive tests
+ * Success       : Yes
+ * Date          : 20th April 2012
+ */
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.3.query.aql
new file mode 100644
index 0000000..9e5a72e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with7/ends-with7.3.query.aql
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+ * Testcase Name : endwith02.aql
+ * Description   : Positive tests
+ * Success       : Yes
+ * Date          : 20th April 2012
+ */
+
+
+for $a in [ends-with("aBCDEFghIa",codepoint-to-string([0041])),
+ends-with("AbCDEFghIA",codepoint-to-string([0041])),
+ends-with("AbCdEfGhIjKlMnOpQrStUvWxYz","xYz"),
+ends-with("abcdef",lowercase("ABCDEf")),
+ends-with("abcdef","abcdef"),
+ends-with("abcdef123","ef123")]
+return $a

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.1.ddl.aql
new file mode 100644
index 0000000..28a3cfb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.1.ddl.aql
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+/*
+ * Testcase Name : endwith03.aql
+ * Description   : Positive tests
+ * Success       : Yes
+ * Date          : 20th April 2012
+ */
+
+// create internal dataset, insert string data into string field and pass the string filed as input to end-with function
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type TestType as {
+name:string
+}
+
+create dataset testds(TestType) primary key name;

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.2.update.aql
new file mode 100644
index 0000000..a42c36d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.2.update.aql
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+/*
+ * Testcase Name : endwith03.aql
+ * Description   : Positive tests
+ * Success       : Yes
+ * Date          : 20th April 2012
+ */
+
+// create internal dataset, insert string data into string field and pass the string filed as input to end-with function
+
+use dataverse test;
+
+insert into dataset testds({"name":"Jim Jones"});
+insert into dataset testds({"name":"Ravi Kumar"});
+insert into dataset testds({"name":"Bruce Li"});
+insert into dataset testds({"name":"Marian Jones"});
+insert into dataset testds({"name":"Phil Jones"});
+insert into dataset testds({"name":"I am Jones"});
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.3.query.aql
new file mode 100644
index 0000000..c1894ba
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/string/ends-with8/ends-with8.3.query.aql
@@ -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.
+ */
+/*
+ * Testcase Name : endwith03.aql
+ * Description   : Positive tests
+ * Success       : Yes
+ * Date          : 20th April 2012
+ */
+
+// create internal dataset, insert string data into string field and pass the string filed as input to ends-with function
+
+use dataverse test;
+
+for $l in dataset('testds')
+order by $l.name
+where ends-with($l.name,"Jones")
+return $l
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.1.ddl.aql
deleted file mode 100644
index d17ea60..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.1.ddl.aql
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 test if exists;
-create dataverse test;
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.2.update.aql
deleted file mode 100644
index 042f3ce..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.2.update.aql
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.
- */

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.3.query.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.3.query.aql
deleted file mode 100644
index 9802279..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/ends-with_01/ends-with_01.3.query.aql
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * 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 dataverse test;
-
-for $x in ["foofoo"]
-for $y in ["barbar"]
-return [ends-with($x, "ar"), ends-with($y, "ar")]

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.1.ddl.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.1.ddl.aql
deleted file mode 100644
index a8f38a9..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.
- */
-/*
- * Testcase Name : endwith02.aql
- * Description   : Positive tests
- * Success       : Yes
- * Date          : 20th April 2012
- */
-

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.2.update.aql
----------------------------------------------------------------------
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.2.update.aql
deleted file mode 100644
index a8f38a9..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/string/endwith02/endwith02.2.update.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * 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.
- */
-/*
- * Testcase Name : endwith02.aql
- * Description   : Positive tests
- * Success       : Yes
- * Date          : 20th April 2012
- */
-