You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/01/30 22:52:42 UTC

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

Dmitry Lychagin has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................

[ASTERIXDB-2272][FUN] Count character positions from 0

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

Details:
- Count character positions in strings,binary from 0 instead of 1
- Affects following built-in functions:
  position(), regexp_position(), substr()/subtring(),
  sub_binary(), find-binary()
- Introduce new configuration property “compiler.stringoffset”
  for backward compatibility
  compiler.stringoffset = 0 - count from 0 (new default)
  compiler.stringoffset = 1 - count from 1 (backwards-compatible setting)

Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
A asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
A asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
M asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
A asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
M asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
M asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
109 files changed, 871 insertions(+), 246 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 812f3c9..02d79c4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -20,6 +20,8 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
@@ -64,7 +66,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = resolveFunction(expr, env);
+        IFunctionDescriptor fd = resolveFunction(expr, env, context);
         switch (fd.getFunctionDescriptorTag()) {
             case SERIALAGGREGATE:
                 return null;
@@ -82,7 +84,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return resolveFunction(expr, env).createRunningAggregateEvaluatorFactory(args);
+        return resolveFunction(expr, env, context).createRunningAggregateEvaluatorFactory(args);
     }
 
     @Override
@@ -90,7 +92,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return resolveFunction(expr, env).createUnnestingEvaluatorFactory(args);
+        return resolveFunction(expr, env, context).createUnnestingEvaluatorFactory(args);
     }
 
     @Override
@@ -138,7 +140,7 @@
         IFunctionDescriptor fd = expr.getFunctionInfo() instanceof IExternalFunctionInfo
                 ? ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor(
                         (IExternalFunctionInfo) expr.getFunctionInfo(), (ICcApplicationContext) context.getAppContext())
-                : resolveFunction(expr, env);
+                : resolveFunction(expr, env, context);
         return fd.createEvaluatorFactory(args);
     }
 
@@ -166,14 +168,14 @@
             AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
             JobGenContext context) throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = resolveFunction(expr, env);
+        IFunctionDescriptor fd = resolveFunction(expr, env, context);
 
         switch (fd.getFunctionDescriptorTag()) {
             case AGGREGATE: {
                 if (BuiltinFunctions.isAggregateFunctionSerializable(fd.getIdentifier())) {
                     AggregateFunctionCallExpression serialAggExpr = BuiltinFunctions
                             .makeSerializableAggregateFunctionExpression(fd.getIdentifier(), expr.getArguments());
-                    IFunctionDescriptor afdd = resolveFunction(serialAggExpr, env);
+                    IFunctionDescriptor afdd = resolveFunction(serialAggExpr, env, context);
                     return afdd.createSerializableAggregateEvaluatorFactory(args);
                 } else {
                     throw new AlgebricksException(
@@ -192,13 +194,14 @@
         }
     }
 
-    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env)
-            throws AlgebricksException {
+    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env,
+            JobGenContext context) throws AlgebricksException {
         FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
         IFunctionDescriptor fd = functionManager.lookupFunction(fnId);
         IFunctionTypeInferer fnTypeInfer = functionManager.lookupFunctionTypeInferer(fnId);
         if (fnTypeInfer != null) {
-            fnTypeInfer.infer(expr, fd, env);
+            CompilerProperties compilerProps = ((IApplicationContext) context.getAppContext()).getCompilerProperties();
+            fnTypeInfer.infer(expr, fd, env, compilerProps);
         }
         return fd;
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
new file mode 100644
index 0000000..4b7d1dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Overrides string offset to be 1 instead of 0
+ */
+@RunWith(Parameterized.class)
+public class StringOffsetTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-stringoffset.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameterized.Parameters(name = "StringOffsetTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_stringoffset.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public StringOffsetTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf b/asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
new file mode 100644
index 0000000..c81deec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
@@ -0,0 +1,60 @@
+; 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.
+
+; Overrides string offset to be 1 instead of 0
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=48MB
+storage.memorycomponent.numpages=16
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.stringoffset=1
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql b/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
index 31087a8..646a2d4 100644
--- a/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
+++ b/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
@@ -18,5 +18,5 @@
  */
 for $c in recordset(['cid', 'int32', 'name', 'string', 'age', 'int32'], 'osfiles', ['asterix_nc1', 'data/spj01/cust1.adm'], ['asterix_nc2', 'data/spj01/cust2.adm'])
 where some $c2 in [ $c ]
-      satisfies substring($c2.name, 1, 1) = "J"
+      satisfies substring($c2.name, 0, 1) = "J"
 return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
index 789cbe0..86e5d4c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
@@ -72,12 +72,12 @@
 
 create dataset Orders(OrderType)
   primary key o_orderkey;
-create dataset Customer(CustomerType) 
+create dataset Customer(CustomerType)
   primary key c_custkey;
 
 declare function q22_customer_tmp() {
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $phone_substr = '13'
     or $phone_substr = '31'
     or $phone_substr = '23'
@@ -94,7 +94,7 @@
 
 let $avg := avg(
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $c.c_acctbal > 0.00
     and ($phone_substr = '13'
     or $phone_substr = '31'
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
index 74573b9..2bbc134 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
@@ -55,7 +55,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
index 0dfdea0..7c8f918 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
@@ -55,7 +55,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
index 7b49736..22bff52 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
@@ -82,14 +82,14 @@
 (
     select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
     from  Customer as c
-    with  phone_substr as tpch.substring(c.c_phone,1,2)
+    with  phone_substr as tpch.substring(c.c_phone,0,2)
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
 with  avg as tpch.coll_avg((
       select element c.c_acctbal
       from  Customer as c
-      with  phone_substr as tpch.substring(c.c_phone,1,2)
+      with  phone_substr as tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
 select element {'cntrycode':cntrycode,'numcust':tpch.coll_count(g),'totacctbal':tpch.coll_sum((
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
index 11df55d..62f92a7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
@@ -83,7 +83,7 @@
           Variable [ Name=$c ]
           Field=c_phone
         ]
-        LiteralExpr [LONG] [1]
+        LiteralExpr [LONG] [0]
         LiteralExpr [LONG] [2]
       ]
     Where
@@ -156,7 +156,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       Where
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
index 58f6a96..6e64f16 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
@@ -19,13 +19,13 @@
 use dataverse test;
 
 let $x := hex("aabbccddaa")
-let $r1 := find-binary($x, hex('')) = 1
-let $r2 := find-binary($x, hex('aa')) = 1
-let $r3 := find-binary($x, hex('aa'), 1) = 5
+let $r1 := find-binary($x, hex('')) = 0
+let $r2 := find-binary($x, hex('aa')) = 0
+let $r3 := find-binary($x, hex('aa'), 1) = 4
 let $r4 := find-binary($x, hex('aabb'), 0) = find-binary($x, hex('aabb'))
-let $r5 := find-binary($x, hex('11')) = 0
-let $r6 := find-binary($x, hex('ccddaa')) = 3
-let $r7 := find-binary($x, hex('ccddaabb')) = 0
+let $r5 := find-binary($x, hex('11')) = -1
+let $r6 := find-binary($x, hex('ccddaa')) = 2
+let $r7 := find-binary($x, hex('ccddaabb')) = -1
 
 let $r8 := find-binary($x, null)
 let $r9 := find-binary(null, null)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
index d4dce94..79980af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
@@ -24,18 +24,18 @@
 let $r2 := sub-binary(hex(''),1) = hex('')
 let $r3 := sub-binary(hex(''),-1) = hex('')
 
-let $r4 := sub-binary($x, 1, binary-length($x)) = $x
-let $r5 := sub-binary($x, 2, 1) = hex('bb')
-let $r6 := sub-binary($x, 2) = hex('bbccdd')
-let $r7 := sub-binary($x, 5, 0) = hex('')
-let $r8 := sub-binary($x, 4, 1) = hex('dd')
-let $r9 := sub-binary($x, 2, 2) = hex('bbcc') 
+let $r4 := sub-binary($x, 0, binary-length($x)) = $x
+let $r5 := sub-binary($x, 1, 1) = hex('bb')
+let $r6 := sub-binary($x, 1) = hex('bbccdd')
+let $r7 := sub-binary($x, 4, 0) = hex('')
+let $r8 := sub-binary($x, 3, 1) = hex('dd')
+let $r9 := sub-binary($x, 1, 2) = hex('bbcc')
 
 let $r10 := sub-binary($x, 0) = $x
 let $r11 := sub-binary($x, -1) = $x
-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('')
+let $r12 := sub-binary($x, 0, 256) = $x
+let $r13 := sub-binary($x, 1, 256) = hex('bbccdd')
+let $r14 := sub-binary($x, 1, -1) = hex('')
 
 return  [ $r1 ,$r2 ,$r3 ,$r4 ,$r5 , $r6 , $r7 , $r8 , $r9 , $r10 , $r11 , $r12 , $r13 , $r14 ]
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
index 0c20cec..bc9e7f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
@@ -24,13 +24,13 @@
  */
 
 let $str1:="Hello World"
-let $str2:=substring($str1,10)
+let $str2:=substring($str1,9)
 
 let $str3:="This is a test string"
-let $str4:=substring($str3,21)
+let $str4:=substring($str3,20)
 
 let $str5:="This is a test string"
-let $str6:=substring($str5,22)
+let $str6:=substring($str5,21)
 
 let $str7:="This is a test string"
 let $str8:=substring($str7,0)
@@ -40,7 +40,7 @@
 
 let $str11:="This is a test string"
 let $str12:="This is a another test string"
-let $str13:=substring(string-concat([$str11,$str12]),21)
+let $str13:=substring(string-concat([$str11,$str12]),20)
 
-let $str14:=substring("UC Irvine",string-length("UC Irvine")/2)
+let $str14:=substring("UC Irvine",string-length("UC Irvine")/2 - 1)
 return { "str2":$str2,"str4":$str4,"str6":$str6,"str8":$str8,"str10":$str10,"str13":$str13,"str14":$str14}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
index ee98756..1c31ea0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
@@ -23,13 +23,13 @@
  * Date           : 18th April 2012
  */
 
-for $a in [ substring("hello world", 7, 5),
-substring("hello world", 1, 11),
-substring("hello world", 3, 9),
-substring("ABCD", 3, 2),
-substring("ABCD", 1, 4),
-substring("UC Irvine", 4, string-length("UC Irvine") - 3),
-substring("UC Irvine", 1, string-length("UC Irvine")),
-substring(substring("UC Irvine", 4), 1, string-length("Irvine"))
+for $a in [ substring("hello world", 6, 5),
+substring("hello world", 0, 11),
+substring("hello world", 2, 9),
+substring("ABCD", 2, 2),
+substring("ABCD", 0, 4),
+substring("UC Irvine", 3, string-length("UC Irvine") - 3),
+substring("UC Irvine", 0, string-length("UC Irvine")),
+substring(substring("UC Irvine", 3), 0, string-length("Irvine"))
 ]
 return $a
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
index cd6ab0a..d500ac5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
@@ -29,4 +29,4 @@
 
 for $a in dataset('testdst')
 order by $a.name
-return substring($a.name, 4, string-length($a.name) - 3);
+return substring($a.name, 3, string-length($a.name) - 3);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
index 0f41b6e..078fd0c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
@@ -28,4 +28,4 @@
 
 for $a in dataset('testdst')
 order by $a.name
-return substring($a.name,4);
+return substring($a.name,3);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
index 57ddd9a..26337e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
@@ -18,5 +18,5 @@
  */
 use dataverse test;
 
-let $c1 := substring("HEllow",2)
+let $c1 := substring("HEllow",1)
 return {"result1": $c1}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
index e230f91..385db08 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
@@ -18,5 +18,5 @@
  */
 use dataverse test;
 
-let $c1 := substring("HEllow",10)
+let $c1 := substring("HEllow",9)
 return {"result1": $c1}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
index dc94226..752ec61 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
@@ -19,4 +19,4 @@
 use dataverse test;
 
 for $x in ["foobar"]
-return substring($x, 2, 3)
+return substring($x, 1, 3)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
index e8cf4e8..ae5f99a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
@@ -23,7 +23,7 @@
   select {
     "c_acctbal": $c.c_acctbal,
     "c_custkey": $c.c_custkey,
-    "cntrycode": substring($c.c_phone, 1, 2)
+    "cntrycode": substring($c.c_phone, 0, 2)
   }
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
index 0c7f979..365e84a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
@@ -23,7 +23,7 @@
   return {
     "c_acctbal": $c.c_acctbal,
     "c_custkey": $c.c_custkey,
-    "cntrycode": substring($c.c_phone, 1, 2)
+    "cntrycode": substring($c.c_phone, 0, 2)
   }
 }
 
@@ -37,7 +37,7 @@
 group by $cntrycode := $ct.cntrycode with $ct
 order by $cntrycode
 return {
-  "cntrycode": $cntrycode, 
-  "numcust": count($ct), 
+  "cntrycode": $cntrycode,
+  "numcust": count($ct),
   "totacctbal": sum(for $i in $ct return $i.c_acctbal)
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
index 4aa0e13..24db233 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
@@ -27,7 +27,7 @@
 
 declare function q22_customer_tmp() {
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $phone_substr = '13'
     or $phone_substr = '31'
     or $phone_substr = '23'
@@ -44,7 +44,7 @@
 
 let $avg := avg(
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $c.c_acctbal > 0.00
     and ($phone_substr = '13'
     or $phone_substr = '31'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
index 2b596c6..1660974 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
@@ -19,5 +19,18 @@
 
 use test;
 
-
-[(test.`find-binary`(test.hex('aabbccddaa'),test.hex('')) = 1),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa')) = 1),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa'),1) = 5),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'),0) = test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'))),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('11')) = 0),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaa')) = 3),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaabb')) = 0),test.`find-binary`(test.hex('aabbccddaa'),null),test.`find-binary`(null,null),test.`find-binary`(null,test.hex('aabbccddaa'))];
+[
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('')) = 0),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa')) = 0),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa'),1) = 4),
+  (
+    test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'),-1) =
+    test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'))
+  ),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('11')) = -1),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaa')) = 2),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaabb')) = -1),
+  test.`find-binary`(test.hex('aabbccddaa'),null),
+  test.`find-binary`(null,null),
+  test.`find-binary`(null,test.hex('aabbccddaa'))
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
index a1141b5..99ef6fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
@@ -20,4 +20,19 @@
 use test;
 
 
-[(test.`sub-binary`(test.hex(''),0) = test.hex('')),(test.`sub-binary`(test.hex(''),1) = test.hex('')),(test.`sub-binary`(test.hex(''),-1) = test.hex('')),(test.`sub-binary`(test.hex('aabbccdd'),1,test.`binary-length`(test.hex('aabbccdd'))) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),2,1) = test.hex('bb')),(test.`sub-binary`(test.hex('aabbccdd'),2) = test.hex('bbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),5,0) = test.hex('')),(test.`sub-binary`(test.hex('aabbccdd'),4,1) = test.hex('dd')),(test.`sub-binary`(test.hex('aabbccdd'),2,2) = test.hex('bbcc')),(test.`sub-binary`(test.hex('aabbccdd'),0) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),-1) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),1,256) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),2,256) = test.hex('bbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),2,-1) = test.hex(''))];
+[
+    (test.`sub-binary`(test.hex(''),0) = test.hex('')),
+    (test.`sub-binary`(test.hex(''),1) = test.hex('')),
+    (test.`sub-binary`(test.hex(''),-1) = test.hex('')),
+    (test.`sub-binary`(test.hex('aabbccdd'),0,test.`binary-length`(test.hex('aabbccdd'))) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,1) = test.hex('bb')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1) = test.hex('bbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),4,0) = test.hex('')),
+    (test.`sub-binary`(test.hex('aabbccdd'),3,1) = test.hex('dd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,2) = test.hex('bbcc')),
+    (test.`sub-binary`(test.hex('aabbccdd'),0) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),-1) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),0,256) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,256) = test.hex('bbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,-1) = test.hex(''))
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
index a4166e5..25017f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
@@ -21,5 +21,5 @@
 
 EXPLAIN
 FROM Employee e
-GROUP BY substr(e.dept.department_id, 1)
-SELECT substr(e.dept.department_id, 1) as deptId, SUM(e.salary) AS star_cost;
+GROUP BY substr(e.dept.department_id, 0)
+SELECT substr(e.dept.department_id, 0) as deptId, SUM(e.salary) AS star_cost;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
index e170a6e..05179af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
@@ -21,5 +21,5 @@
 
 EXPLAIN
 FROM Employee e
-GROUP BY substr(e.department_id, 1)
-SELECT substr(e.department_id, 1) as deptId, SUM(e.salary) AS star_cost;
+GROUP BY substr(e.department_id, 0)
+SELECT substr(e.department_id, 0) as deptId, SUM(e.salary) AS star_cost;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
index 5c6cc38..8e8ce59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
@@ -22,6 +22,6 @@
 FROM Employee e
     JOIN Incentive i ON e.job_category = i.job_category
     JOIN SuperStars s ON e.id = s.id
-GROUP BY substr(e.department_id, 1)
-SELECT substr(e.department_id, 1), SUM(e.salary + i.bonus);
+GROUP BY substr(e.department_id, 0)
+SELECT substr(e.department_id, 0), SUM(e.salary + i.bonus);
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
index c2d046d..eb6fa1d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
@@ -22,6 +22,6 @@
 FROM Employee e
     JOIN Incentive i ON e.job_category = i.job_category
     JOIN SuperStars s ON e.id = s.id
-GROUP BY substr(e.department_id, 1)
-SELECT substr(e.department_id, 1) as deptId, SUM(e.salary + i.bonus) AS star_cost;
+GROUP BY substr(e.department_id, 0)
+SELECT substr(e.department_id, 0) as deptId, SUM(e.salary + i.bonus) AS star_cost;
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
index d6bb472..10e525f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
@@ -18,24 +18,24 @@
  */
 
 {
-  'a': SUBSTR(null, 1),
+  'a': SUBSTR(null, 0),
   'b': SUBSTR('abc', null),
   'c': SUBSTR(null, null),
-  'd': SUBSTR(null, 1, 2),
+  'd': SUBSTR(null, 0, 2),
   'e': substr('abc', null, 2),
-  'f': SUBSTR('abc', 1, null),
+  'f': SUBSTR('abc', 0, null),
   'g': SUBSTR(null, null, 2),
-  'h': substr(null, 1, null),
+  'h': substr(null, 0, null),
   'i': SUBSTR(null, null, null),
   'j': SUBSTR('abc', null, null),
-  'k': SUBSTR(missing, 1),
+  'k': SUBSTR(missing, 0),
   'l': SUBSTR('abc', missing),
   'm': SUBSTR(missing, missing),
-  'o': SUBSTR(missing, 1, 2),
+  'o': SUBSTR(missing, 0, 2),
   'p': substr('abc', missing, 2),
-  'q': SUBSTR('abc', 1, missing),
+  'q': SUBSTR('abc', 0, missing),
   'r': SUBSTR(missing, missing, 2),
-  's': substr(missing, 1, missing),
+  's': substr(missing, 0, missing),
   't': SUBSTR(missing, missing, missing),
   'u': SUBSTR('abc', missing, missing),
   'v': SUBSTR(null, missing, missing),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
index 3fa0247..b9c79d0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
@@ -23,4 +23,4 @@
  * Date           : 18th April 2012
  */
 
-{'str2':substring('Hello World',10),'str4':substring('This is a test string',21),'str6':substring('This is a test string',22),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-1),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),21),'str14':substring('UC Irvine',(`string-length`('UC Irvine') / 2))};
+{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-1),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') / 2 - 1))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
index a088506..860173e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
@@ -24,5 +24,5 @@
  */
 
 select element a
-from  [substring('hello world',7,5),substring('hello world',1,11),substring('hello world',3,9),substring('ABCD',3,2),substring('ABCD',1,4),substring('UC Irvine',4,(`string-length`('UC Irvine') - 3)),substring('UC Irvine',1,`string-length`('UC Irvine')),substring(substring('UC Irvine',4),1,`string-length`('Irvine'))] as a
+from  [substring('hello world',6,5),substring('hello world',0,11),substring('hello world',2,9),substring('ABCD',2,2),substring('ABCD',0,4),substring('UC Irvine',3,(`string-length`('UC Irvine') - 3)),substring('UC Irvine',0,`string-length`('UC Irvine')),substring(substring('UC Irvine',3),0,`string-length`('Irvine'))] as a
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
index 2bfa797..9926926 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
@@ -26,7 +26,7 @@
 use test;
 
 
-select element test.substring(a.name,4,(test.`string-length`(a.name) - 3))
+select element test.substring(a.name,3,(test.`string-length`(a.name) - 3))
 from  testdst as a
 order by a.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
index 991a911..dfd8ebb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.substring(a.name,4)
+select element test.substring(a.name,3)
 from  testdst as a
 order by a.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
index 27ff9ae..fa3b8af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
@@ -20,4 +20,4 @@
 use test;
 
 
-{'result1':test.substring('HEllow',2)};
+{'result1':test.substring('HEllow',1)};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
index eee0126..9f1fbc6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
@@ -20,4 +20,4 @@
 use test;
 
 
-{'result1':test.substring('HEllow',10)};
+{'result1':test.substring('HEllow',9)};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
index 40a6960..1ad0a0f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
@@ -20,6 +20,6 @@
 use test;
 
 
-select element test.substring(x,2,3)
+select element test.substring(x,1,3)
 from  ['foobar'] as x
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
new file mode 100644
index 0000000..073220f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),-1),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),0),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),1),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),2),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('1122'))
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp
new file mode 100644
index 0000000..7ea703e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    find_binary(hex('aabbccddeeff'),hex('aabb')),
+    find_binary(hex('aabbccddeeff'),hex('ccdd')),
+    find_binary(hex('aabbccddeeff'),hex('1122'))
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp
new file mode 100644
index 0000000..8f321c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    position('abcdefg','ab'),
+    position('abcdefg','de'),
+    position('abc', 'de')
+]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp
new file mode 100644
index 0000000..7a27b2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+  REGEXP_POSITION('mnop','.'),
+  REGEXP_POSITION('mnop','p'),
+  REGEXP_POSITION('abcdefABCDEF','/d')
+]
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp
new file mode 100644
index 0000000..ac47eb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+  REGEXP_POSITION('mnop','.','x'),
+  REGEXP_POSITION('mnop','p','x'),
+  REGEXP_POSITION('abcdefABCDEF','/d','x')
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
new file mode 100644
index 0000000..6989cf7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    print_binary(sub_binary(hex('aabbccddeeff'),-1,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),0,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),1,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),2,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),3,2), "hex")
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
new file mode 100644
index 0000000..075f803
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    print_binary(sub_binary(hex('aabbccddeeff'),-1), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),0), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),1), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),2), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),3), 'hex')
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
new file mode 100644
index 0000000..1c35e7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    substring('abcdefg',-1,2),
+    substring('abcdefg',0,2),
+    substring('abcdefg',1,2),
+    substring('abcdefg',2,2),
+    substring('abcdefg',3,2)
+]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
new file mode 100644
index 0000000..65d10b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    substring('abcdefg',-1),
+    substring('abcdefg',0),
+    substring('abcdefg',1),
+    substring('abcdefg',2),
+    substring('abcdefg',3)
+]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
index e7328f3..cdaad5e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
@@ -21,7 +21,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 ),
 avg AS (
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
index c47aad3..53d2a68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
@@ -21,7 +21,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
index 66e99ab..2f46ea7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
@@ -23,7 +23,7 @@
 
 SELECT  user.name AS uname,
         user.alias AS alias,
-        substring(message.message, 30) AS msg
+        substring(message.message, 29) AS msg
 FROM FacebookMessages AS message,
      FacebookUsers AS user
 WHERE message.`author-id` = user.id and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
index 7f84895..a8f43a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
@@ -21,7 +21,7 @@
 
 SELECT  user.name AS uname,
         user.alias AS alias,
-        substring(message.message, 30) AS msg
+        substring(message.message, 29) AS msg
 FROM FacebookMessages AS message,
      FacebookUsers AS user
 WHERE message.`author-id` = user.id and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
index 484757a..7152cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
@@ -28,7 +28,7 @@
     ,date_dim d
 WHERE cs.cs_bill_customer_sk = c.c_customer_sk
 AND c.c_current_addr_sk = ca.ca_address_sk
-AND ( substr(ca.ca_zip,1,5) in ['85669', '86197','88274','83405','86475',
+AND ( substr(ca.ca_zip,0,5) in ['85669', '86197','88274','83405','86475',
                              '85392', '85460', '80348', '81792']
 OR ca.ca_state in ['CA','WA','GA']
 OR cs.cs_sales_price > 500)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
index 5e3b339..52c298e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
@@ -30,7 +30,7 @@
 AND d.d_year=1999
 AND ss.ss_customer_sk = c.c_customer_sk
 AND c.c_current_addr_sk = ca.ca_address_sk
-AND substr(ca.ca_zip,1,5) != substr(s.s_zip,1,5)
+AND substr(ca.ca_zip,0,5) != substr(s.s_zip,0,5)
 AND ss.ss_store_sk = s.s_store_sk
 GROUP BY i.i_brand
         ,i.i_brand_id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
index d2a0d85..2d8fdc9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
@@ -24,7 +24,7 @@
  WHERE ws.ws_bill_customer_sk = c.c_customer_sk
   AND c.c_current_addr_sk = ca.ca_address_sk
   AND ws.ws_item_sk = i2.i_item_sk
-  AND ( substr(ca.ca_zip,1,5) IN ['85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792']
+  AND ( substr(ca.ca_zip,0,5) IN ['85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792']
         OR
         i2.i_item_id IN (SELECT VALUE i1.i_item_id
                          FROM item i1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
index b328964..6f8eb1e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
@@ -21,7 +21,7 @@
 USE tpcds;
 
 SELECT
-   SUBSTR(w.w_warehouse_name,1,20)
+   SUBSTR(w.w_warehouse_name,0,20)
   ,sm.sm_type
   ,web.web_name
   ,SUM(CASE WHEN (ws.ws_ship_date_sk - ws.ws_sold_date_sk) <= 30 THEN 1 ELSE 0 END)  AS c30_days
@@ -45,10 +45,10 @@
 AND ws.ws_ship_mode_sk   = sm.sm_ship_mode_sk
 AND ws.ws_web_site_sk    = web.web_site_sk
 GROUP BY
-   SUBSTR(w.w_warehouse_name,1,20)
+   SUBSTR(w.w_warehouse_name,0,20)
   ,sm.sm_type
   ,web.web_name
-ORDER BY SUBSTR(w.w_warehouse_name,1,20)
+ORDER BY SUBSTR(w.w_warehouse_name,0,20)
         ,sm_type
        ,web_name
 LIMIT 100;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
index 0bd5649..ea27bdf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
@@ -21,7 +21,7 @@
 USE tpcds;
 
 SELECT
-  c.c_last_name,c.c_first_name,substr(ms.s_city,1,30),ms.ss_ticket_number,ms.amt,ms.profit
+  c.c_last_name,c.c_first_name,substr(ms.s_city,0,30),ms.ss_ticket_number,ms.amt,ms.profit
   FROM
    (SELECT ss_ticket_number
           ,ss_customer_sk
@@ -38,5 +38,5 @@
     AND store.s_number_employees >= 200 AND store.s_number_employees <= 295
     group by store_sales.ss_ticket_number,store_sales.ss_customer_sk,store_sales.ss_addr_sk,store.s_city) ms,customer c
     WHERE ms.ss_customer_sk = c.c_customer_sk
- ORDER BY c.c_last_name,c.c_first_name,SUBSTR(ms.s_city,1,30), ms.profit
+ ORDER BY c.c_last_name,c.c_first_name,SUBSTR(ms.s_city,0,30), ms.profit
  LIMIT 100;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
index 8ab3f25..7951333 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
@@ -20,7 +20,7 @@
 
 use tpcds;
 
-select  substr(r_reason_desc,1,20)
+select  substr(r_reason_desc,0,20)
        ,avg(ws.ws_quantity)
        ,avg(wr.wr_refunded_cash)
        ,avg(wr.wr_fee)
@@ -96,7 +96,7 @@
     )
    )
 group by r.r_reason_desc
-order by substr(r_reason_desc,1,20)
+order by substr(r_reason_desc,0,20)
         ,avg(ws.ws_quantity)
         ,avg(wr.wr_refunded_cash)
         ,avg(wr.wr_fee)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index bdd7323..d33a095 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,1,2)}
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,0,2)}
     from  Customer as c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index f873dc6..a3a4473 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -21,7 +21,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
index 4a23a50..c6afe18 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
@@ -30,7 +30,7 @@
 (
     SELECT c.c_acctbal, c.c_custkey, phone_substr AS cntrycode
     FROM  Customer c
-    LET  phone_substr = substring(c_phone,1,2)
+    LET  phone_substr = substring(c_phone,0,2)
     WHERE phone_substr = '13' OR phone_substr = '31' OR phone_substr = '23' OR phone_substr = '29'
           OR phone_substr = '30' OR phone_substr = '18' OR phone_substr = '17'
 )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index 109e5a9..206033a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    SELECT c.c_acctbal AS c_acctbal, c.c_custkey AS c_custkey, substring(c.c_phone,1,2) AS cntrycode
+    SELECT c.c_acctbal AS c_acctbal, c.c_custkey AS c_custkey, substring(c.c_phone,0,2) AS cntrycode
     FROM  Customer AS c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
index 3c99324..531e17f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
@@ -30,7 +30,7 @@
 (
     SELECT c.c_acctbal AS c_acctbal, c.c_custkey AS c_custkey, phone_substr AS cntrycode
     FROM  Customer AS c
-    WITH  phone_substr AS substring(c.c_phone,1,2)
+    WITH  phone_substr AS substring(c.c_phone,0,2)
     WHERE phone_substr = '13' OR phone_substr = '31' OR phone_substr = '23' OR phone_substr = '29'
           OR phone_substr = '30' OR phone_substr = '18' OR phone_substr = '17'
 )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index bdd7323..d33a095 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,1,2)}
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,0,2)}
     from  Customer as c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
index dc54947..4cd5d54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
@@ -30,14 +30,14 @@
 (
     select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
     from  Customer as c
-    let  phone_substr = tpch.substring(c.c_phone,1,2)
+    let  phone_substr = tpch.substring(c.c_phone,0,2)
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
 with  avg as tpch.coll_avg((
       select element c.c_acctbal
       from  Customer as c
-      let  phone_substr = tpch.substring(c.c_phone,1,2)
+      let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
 select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index bdd7323..d33a095 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,1,2)}
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,0,2)}
     from  Customer as c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
index dc54947..4cd5d54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
@@ -30,14 +30,14 @@
 (
     select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
     from  Customer as c
-    let  phone_substr = tpch.substring(c.c_phone,1,2)
+    let  phone_substr = tpch.substring(c.c_phone,0,2)
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
 with  avg as tpch.coll_avg((
       select element c.c_acctbal
       from  Customer as c
-      let  phone_substr = tpch.substring(c.c_phone,1,2)
+      let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
 select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
index 5f25e64..98cedcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
@@ -28,7 +28,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   project ([$$34, $$37])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$37] <- [substring($$41.getField("department_id"), 1)]
+                    assign [$$37] <- [substring($$41.getField("department_id"), 0)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$34, $$41])
                       -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
index 92df490..e2e1c91 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
@@ -28,7 +28,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   project ([$$33, $$36])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$36, $$33] <- [substring($$e.getField(1), 1), $$e.getField(2)]
+                    assign [$$36, $$33] <- [substring($$e.getField(1), 0), $$e.getField(2)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$e])
                       -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
index 5c54d04..76e3e97 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
@@ -1 +1 @@
-[ 3, -1 ]
+[ 2, -1 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
index 83ee615..428917f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
@@ -1,15 +1,15 @@
-1
+0
 -1
-1
-1
-1
+0
+0
+0
 -1
-1
+0
 -1
 -1
-1
-1
-1
-1
+0
+0
+0
+0
 -1
-1
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
index f962896..fa99af7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
@@ -1 +1 @@
-{ "result1": 1 }
+{ "result1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
new file mode 100644
index 0000000..0fae953
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
@@ -0,0 +1 @@
+[ 1, 1, 1, 7, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
new file mode 100644
index 0000000..299ae43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
@@ -0,0 +1 @@
+[ 1, 3, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
new file mode 100644
index 0000000..613f89b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
@@ -0,0 +1 @@
+[ 1, 4, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
new file mode 100644
index 0000000..613f89b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
@@ -0,0 +1 @@
+[ 1, 4, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
new file mode 100644
index 0000000..613f89b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
@@ -0,0 +1 @@
+[ 1, 4, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
new file mode 100644
index 0000000..f5e809d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
@@ -0,0 +1 @@
+[ "AABB", "AABB", "AABB", "BBCC", "CCDD" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
new file mode 100644
index 0000000..275d27a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
@@ -0,0 +1 @@
+[ "AABBCCDDEEFF", "AABBCCDDEEFF", "AABBCCDDEEFF", "BBCCDDEEFF", "CCDDEEFF" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
new file mode 100644
index 0000000..b92a705
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
@@ -0,0 +1 @@
+[ "ab", "ab", "ab", "bc", "cd" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
new file mode 100644
index 0000000..e341861
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
@@ -0,0 +1 @@
+[ "abcdefg", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
index cd2f25c..813591c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
@@ -11,7 +11,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [1]
+    LiteralExpr [LONG] [0]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@2[
@@ -23,7 +23,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [1]
+    LiteralExpr [LONG] [0]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@3[
@@ -36,7 +36,7 @@
       LiteralExpr [LONG] [1]
     ]
     =
-    LiteralExpr [LONG] [5]
+    LiteralExpr [LONG] [4]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@3[
@@ -46,7 +46,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabb]
       ]
-      LiteralExpr [LONG] [0]
+      - LiteralExpr [LONG] [1]
     ]
     =
     FunctionCall test.find-binary@2[
@@ -68,7 +68,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [0]
+    - LiteralExpr [LONG] [1]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@2[
@@ -80,7 +80,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [3]
+    LiteralExpr [LONG] [2]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@2[
@@ -92,7 +92,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [0]
+    - LiteralExpr [LONG] [1]
   ]
   FunctionCall test.find-binary@2[
     FunctionCall test.hex@1[
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
index 6ec8199..5a232ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
@@ -42,7 +42,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       FunctionCall test.binary-length@1[
         FunctionCall test.hex@1[
           LiteralExpr [STRING] [aabbccdd]
@@ -59,7 +59,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       LiteralExpr [LONG] [1]
     ]
     =
@@ -72,7 +72,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
     ]
     =
     FunctionCall test.hex@1[
@@ -84,7 +84,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [5]
+      LiteralExpr [LONG] [4]
       LiteralExpr [LONG] [0]
     ]
     =
@@ -97,7 +97,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [4]
+      LiteralExpr [LONG] [3]
       LiteralExpr [LONG] [1]
     ]
     =
@@ -110,7 +110,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       LiteralExpr [LONG] [2]
     ]
     =
@@ -147,7 +147,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       LiteralExpr [LONG] [256]
     ]
     =
@@ -160,7 +160,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       LiteralExpr [LONG] [256]
     ]
     =
@@ -173,7 +173,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       - LiteralExpr [LONG] [1]
     ]
     =
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
index 9e8f66d..a86d0a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
@@ -5,7 +5,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [Hello World]
-      LiteralExpr [LONG] [10]
+      LiteralExpr [LONG] [9]
     ]
   )
   (
@@ -13,7 +13,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [This is a test string]
-      LiteralExpr [LONG] [21]
+      LiteralExpr [LONG] [20]
     ]
   )
   (
@@ -21,7 +21,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [This is a test string]
-      LiteralExpr [LONG] [22]
+      LiteralExpr [LONG] [21]
     ]
   )
   (
@@ -50,7 +50,7 @@
           LiteralExpr [STRING] [This is a another test string]
         ]
       ]
-      LiteralExpr [LONG] [21]
+      LiteralExpr [LONG] [20]
     ]
   )
   (
@@ -59,11 +59,15 @@
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [UC Irvine]
       OperatorExpr [
-        FunctionCall null.string-length@1[
-          LiteralExpr [STRING] [UC Irvine]
+        OperatorExpr [
+          FunctionCall null.string-length@1[
+            LiteralExpr [STRING] [UC Irvine]
+          ]
+          /
+          LiteralExpr [LONG] [2]
         ]
-        /
-        LiteralExpr [LONG] [2]
+        -
+        LiteralExpr [LONG] [1]
       ]
     ]
   )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
index ccd395d..f85b949 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
@@ -5,32 +5,32 @@
 FROM [  OrderedListConstructor [
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [hello world]
-      LiteralExpr [LONG] [7]
+      LiteralExpr [LONG] [6]
       LiteralExpr [LONG] [5]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [hello world]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       LiteralExpr [LONG] [11]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [hello world]
-      LiteralExpr [LONG] [3]
+      LiteralExpr [LONG] [2]
       LiteralExpr [LONG] [9]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [ABCD]
-      LiteralExpr [LONG] [3]
+      LiteralExpr [LONG] [2]
       LiteralExpr [LONG] [2]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [ABCD]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       LiteralExpr [LONG] [4]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [UC Irvine]
-      LiteralExpr [LONG] [4]
+      LiteralExpr [LONG] [3]
       OperatorExpr [
         FunctionCall null.string-length@1[
           LiteralExpr [STRING] [UC Irvine]
@@ -41,7 +41,7 @@
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [UC Irvine]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       FunctionCall null.string-length@1[
         LiteralExpr [STRING] [UC Irvine]
       ]
@@ -49,9 +49,9 @@
     FunctionCall null.substring@3[
       FunctionCall null.substring@2[
         LiteralExpr [STRING] [UC Irvine]
-        LiteralExpr [LONG] [4]
+        LiteralExpr [LONG] [3]
       ]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       FunctionCall null.string-length@1[
         LiteralExpr [STRING] [Irvine]
       ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
index d02dca1..685a3b4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
@@ -6,7 +6,7 @@
     Variable [ Name=$a ]
     Field=name
   ]
-  LiteralExpr [LONG] [4]
+  LiteralExpr [LONG] [3]
   OperatorExpr [
     FunctionCall test.string-length@1[
       FieldAccessor [
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
index 4f1d3f4..f27bf4d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
@@ -6,7 +6,7 @@
     Variable [ Name=$a ]
     Field=name
   ]
-  LiteralExpr [LONG] [4]
+  LiteralExpr [LONG] [3]
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
index ed84aaa..c170655 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
@@ -6,7 +6,7 @@
     :
     FunctionCall test.substring@2[
       LiteralExpr [STRING] [HEllow]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
index 865e373..358d135 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
@@ -6,7 +6,7 @@
     :
     FunctionCall test.substring@2[
       LiteralExpr [STRING] [HEllow]
-      LiteralExpr [LONG] [10]
+      LiteralExpr [LONG] [9]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
index cd3b906..d24c5b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
@@ -3,7 +3,7 @@
 SELECT ELEMENT [
 FunctionCall test.substring@3[
   Variable [ Name=$x ]
-  LiteralExpr [LONG] [2]
+  LiteralExpr [LONG] [1]
   LiteralExpr [LONG] [3]
 ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
index 7425817..cdd8a53 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
@@ -16,7 +16,7 @@
     Variable [ Name=$message ]
     Field=message
   ]
-  LiteralExpr [LONG] [30]
+  LiteralExpr [LONG] [29]
 ]
 msg
 ]
@@ -78,7 +78,7 @@
       Variable [ Name=$message ]
       Field=message
     ]
-    LiteralExpr [LONG] [30]
+    LiteralExpr [LONG] [29]
   ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
index 7425817..cdd8a53 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
@@ -16,7 +16,7 @@
     Variable [ Name=$message ]
     Field=message
   ]
-  LiteralExpr [LONG] [30]
+  LiteralExpr [LONG] [29]
 ]
 msg
 ]
@@ -78,7 +78,7 @@
       Variable [ Name=$message ]
       Field=message
     ]
-    LiteralExpr [LONG] [30]
+    LiteralExpr [LONG] [29]
   ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
index 9b94363..0ad2c82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
@@ -27,7 +27,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
index 9b94363..0ad2c82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
@@ -27,7 +27,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
index d9f60e3..928d5e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
@@ -36,7 +36,7 @@
           Variable [ Name=$c ]
           Field=c_phone
         ]
-        LiteralExpr [LONG] [1]
+        LiteralExpr [LONG] [0]
         LiteralExpr [LONG] [2]
       ]
     Where
@@ -109,7 +109,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       Where
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
new file mode 100644
index 0000000..f7da162
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
@@ -0,0 +1,67 @@
+<!--
+ ! 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.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="stringoffset">
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="position">
+        <output-dir compare="Text">position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="regexp_position">
+        <output-dir compare="Text">regexp_position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="regexp_position_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="substring">
+        <output-dir compare="Text">substring</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="substring2">
+        <output-dir compare="Text">substring2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="subbinary">
+        <output-dir compare="Text">subbinary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="subbinary2">
+        <output-dir compare="Text">subbinary2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="findbinary">
+        <output-dir compare="Text">findbinary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="findbinary2">
+        <output-dir compare="Text">findbinary2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 0d29943..39869af 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -57,7 +57,8 @@
                         + "other integer values dictate the number of query execution parallel partitions. The system will "
                         + "fall back to use the number of all available CPU cores in the cluster as the degree of parallelism "
                         + "if the number set by a user is too large or too small"),
-        COMPILER_PREGELIX_HOME(STRING, "~/pregelix", "Pregelix installation root directory");
+        COMPILER_PREGELIX_HOME(STRING, "~/pregelix", "Pregelix installation root directory"),
+        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -132,4 +133,9 @@
     public String getPregelixHome() {
         return accessor.getString(Option.COMPILER_PREGELIX_HOME);
     }
+
+    public int getStringOffset() {
+        int value = accessor.getInt(Option.COMPILER_STRINGOFFSET);
+        return value > 0 ? 1 : 0;
+    }
 }
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
index 95d0550..1f56c4c 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
@@ -211,8 +211,8 @@
     * `string` : a `string` that might contain the pattern,
     * `string_pattern` : a pattern `string` to be matched.
  * Return Value:
-    * the first position that `string_pattern` appears within `string`,
-      or -1 if it does not appear,
+    * the first position that `string_pattern` appears within `string`
+      (starting at 0), or -1 if it does not appear,
     * `missing` if any argument is a `missing` value,
     * `null` if any argument is a `null` value but no argument is a `missing` value,
     * any other non-string input value will cause a type error.
@@ -306,8 +306,8 @@
     * `string_flag` : (Optional) a `string` with flags to be used during regular expression matching.
         * The following modes are enabled with these flags: dotall (s), multiline (m), case_insensitive (i), and comments and whitespace (x).
  * Return Value:
-    * the first position that the regular expression `string_pattern` appears in `string`,
-      or -1 if it does not appear.
+    * the first position that the regular expression `string_pattern` appears in `string`
+      (starting at 0), or -1 if it does not appear.
     * `missing` if any argument is a `missing` value,
     * `null` if any argument is a `null` value but no argument is a `missing` value,
     * any other non-string input value will cause a type error.
@@ -363,7 +363,7 @@
  * Returns a string formed by repeating the input `string` `n` times.
  * Arguments:
     * `string` : a `string` to be repeated,
-    * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`.
+    * `n` : an `tinyint`/`smallint`/`integer`/`bigint` value - how many times the string should be repeated.
  * Return Value:
     * a string that repeats the input `string` `n` times,
     * `missing` if any argument is a `missing` value,
@@ -471,7 +471,8 @@
  * Returns the substring from the given string `string` based on the given start offset `offset` with the optional `length`.
  * Arguments:
     * `string` : a `string` to be extracted,
-    * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`,
+    * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`
+                 (starting at 0),
     * `length` : (Optional) an an `tinyint`/`smallint`/`integer`/`bigint` value as the length of the substring.
  * Return Value:
     * a `string` that represents the substring,
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md b/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
index 2902496..82a68be 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
@@ -97,7 +97,7 @@
   * Arguments:
     * `binary` : a `binary` to be extracted,
     * `offset` : a `tinyint`, `smallint`, `integer`, or `bigint` value
-       as the starting offset of the sub binary in `binary`,
+       as the starting offset of the sub binary in `binary` (starting at 0),
     * `length` : (Optional) a `tinyint`, `smallint`, `integer`, or `bigint` value
                   as the length of the sub binary.
   * Return Value:
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
index b8e27e1..97042e2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
@@ -19,11 +19,12 @@
 
 package org.apache.asterix.om.functions;
 
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 
 public interface IFunctionTypeInferer {
-    void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-            throws AlgebricksException;
+    void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+            CompilerProperties compilerProps) throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java
new file mode 100644
index 0000000..3f8f45f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.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.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+
+public abstract class AbstractStringOffsetConfigurableDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    protected int stringOffset;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        stringOffset = (int) states[0];
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
index 6f4d116..e9ec211 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
@@ -24,7 +24,8 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -32,7 +33,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class StringPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class StringPositionDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@@ -40,12 +41,19 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StringPositionDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
+        }
     };
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
@@ -55,7 +63,7 @@
                     @Override
                     protected int compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
                         int pos = UTF8StringPointable.find(left, right, false);
-                        return pos < 0 ? pos : pos + 1;
+                        return pos < 0 ? pos : pos + baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
index 1bb0fdc..b39c473 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
@@ -24,8 +24,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -33,13 +34,18 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class StringRegExpPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class StringRegExpPositionDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StringRegExpPositionDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -48,6 +54,8 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
@@ -55,11 +63,10 @@
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
-                    protected int compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr)
-                            throws IOException {
+                    protected int compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr) {
                         matcher.build(srcPtr, patternPtr);
                         int pos = matcher.postion();
-                        return pos < 0 ? pos : pos + 1;
+                        return pos < 0 ? pos : pos + baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
index 35eb3b1..16d428f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
@@ -24,8 +24,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -33,7 +34,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class StringRegExpPositionWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class StringRegExpPositionWithFlagDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@@ -41,12 +42,19 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StringRegExpPositionWithFlagDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
+        }
     };
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
@@ -56,10 +64,10 @@
 
                     @Override
                     protected int compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr,
-                            UTF8StringPointable flagPtr) throws IOException {
+                            UTF8StringPointable flagPtr) {
                         matcher.build(srcPtr, patternPtr, flagPtr);
                         int pos = matcher.postion();
-                        return pos < 0 ? pos : pos + 1;
+                        return pos < 0 ? pos : pos + baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
index 5a36942..8ebfb94 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
@@ -26,10 +26,11 @@
 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.ATypeTag;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -43,12 +44,17 @@
 import org.apache.hyracks.data.std.util.UTF8StringBuilder;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class Substring2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class Substring2Descriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new Substring2Descriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -56,6 +62,8 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
@@ -78,7 +86,8 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset) - 1;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset)
+                                - baseOffset;
                         bytes = argString.getByteArray();
                         offset = argString.getStartOffset();
                         int len = argString.getLength();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
index 28bbe08..d2a1203 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
@@ -26,10 +26,11 @@
 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.ATypeTag;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -43,7 +44,7 @@
 import org.apache.hyracks.data.std.util.UTF8StringBuilder;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class SubstringDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class SubstringDescriptor extends AbstractStringOffsetConfigurableDescriptor {
 
     private static final long serialVersionUID = 1L;
 
@@ -52,12 +53,19 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubstringDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
+        }
     };
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
@@ -85,7 +93,8 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset) - 1;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset)
+                                - baseOffset;
 
                         bytes = argLen.getByteArray();
                         offset = argLen.getStartOffset();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
index 827b43e..c88fe25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
@@ -35,8 +35,9 @@
 public abstract class AbstractFindBinaryEvaluator extends AbstractBinaryScalarEvaluator {
 
     private static final ATypeTag[] EXPECTED_INPUT_TAG = { ATypeTag.BINARY, ATypeTag.BINARY };
-    protected String functionName;
-    protected AMutableInt64 result = new AMutableInt64(-1);
+    protected final int baseOffset;
+    protected final String functionName;
+    protected final AMutableInt64 result = new AMutableInt64(-1);
     protected final ByteArrayPointable textPtr = new ByteArrayPointable();
     protected final ByteArrayPointable wordPtr = new ByteArrayPointable();
 
@@ -45,8 +46,9 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
 
     public AbstractFindBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            String functionName) throws HyracksDataException {
+            int baseOffset, String functionName) throws HyracksDataException {
         super(context, copyEvaluatorFactories);
+        this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
 
@@ -64,9 +66,9 @@
         checkTypeMachingThrowsIfNot(functionName, EXPECTED_INPUT_TAG, textTag, wordTag);
         textPtr.set(pointables[0].getByteArray(), pointables[0].getStartOffset() + 1, pointables[0].getLength() - 1);
         wordPtr.set(pointables[1].getByteArray(), pointables[0].getStartOffset() + 1, pointables[1].getLength() - 1);
-        result.setValue(1L + indexOf(textPtr.getByteArray(), textPtr.getContentStartOffset(),
-                textPtr.getContentLength(), wordPtr.getByteArray(), wordPtr.getContentStartOffset(),
-                wordPtr.getContentLength(), fromOffset));
+        int pos = indexOf(textPtr.getByteArray(), textPtr.getContentStartOffset(), textPtr.getContentLength(),
+                wordPtr.getByteArray(), wordPtr.getContentStartOffset(), wordPtr.getContentLength(), fromOffset);
+        result.setValue(pos < 0 ? pos : pos + baseOffset);
         intSerde.serialize(result, dataOutput);
         resultPointable.set(resultStorage);
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
index cd7b7d3..fc049a4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
@@ -36,13 +36,15 @@
 
     private ByteArrayPointable byteArrayPointable = new ByteArrayPointable();
     private byte[] metaBuffer = new byte[5];
+    protected final int baseOffset;
     protected final String functionName;
 
     private static final ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.BINARY, ATypeTag.INTEGER };
 
     public AbstractSubBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            String functionName) throws HyracksDataException {
+            int baseOffset, String functionName) throws HyracksDataException {
         super(context, copyEvaluatorFactories);
+        this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
 
@@ -67,9 +69,8 @@
 
             int subStart;
 
-            // strange SQL index convention
             subStart = ATypeHierarchy.getIntegerValue(BuiltinFunctions.SUBBINARY_FROM.getName(), 1, startBytes, offset)
-                    - 1;
+                    - baseOffset;
 
             int totalLength = byteArrayPointable.getContentLength();
             int subLength = getSubLength(tuple);
@@ -78,7 +79,10 @@
                 subStart = 0;
             }
 
-            if (subStart >= totalLength || subLength < 0) {
+            if (subStart >= totalLength) {
+                subStart = 0;
+                subLength = 0;
+            } else if (subLength < 0) {
                 subLength = 0;
             } else if (subLength > totalLength // for the IntMax case
                     || subStart + subLength > totalLength) {
@@ -88,6 +92,7 @@
             dataOutput.write(ATypeTag.BINARY.serialize());
             int metaLength = VarLenIntEncoderDecoder.encode(subLength, metaBuffer, 0);
             dataOutput.write(metaBuffer, 0, metaLength);
+
             dataOutput.write(byteArrayPointable.getByteArray(), byteArrayPointable.getContentStartOffset() + subStart,
                     subLength);
         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
index 61a24c4..307585d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
@@ -22,7 +22,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -30,12 +32,17 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class FindBinaryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class FindBinaryDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FindBinaryDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -49,11 +56,13 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
-                    protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
+                    protected int getFromOffset(IFrameTupleReference tuple) {
                         return 0;
                     }
                 };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
index 71f8459..243d344 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
@@ -22,8 +22,10 @@
 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.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -31,13 +33,18 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class FindBinaryFromDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class FindBinaryFromDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FindBinaryFromDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -51,13 +58,15 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
                     protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
-                                pointables[2].getByteArray(), pointables[2].getStartOffset());
+                                pointables[2].getByteArray(), pointables[2].getStartOffset()) - baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
index a677072..3b07c6c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
@@ -22,7 +22,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -30,12 +32,17 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class SubBinaryFromDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class SubBinaryFromDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubBinaryFromDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -49,11 +56,13 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractSubBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
-                    protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
+                    protected int getSubLength(IFrameTupleReference tuple) {
                         return Integer.MAX_VALUE;
                     }
                 };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
index d91c0f4..961edbf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
@@ -22,8 +22,10 @@
 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.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -31,12 +33,17 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class SubBinaryFromToDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class SubBinaryFromToDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubBinaryFromToDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -50,10 +57,12 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractSubBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
                     protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index 00a5ec8..b8d9778 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -19,6 +19,7 @@
 
 package org.apache.asterix.runtime.functions;
 
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -53,16 +54,24 @@
 
     public static final IFunctionTypeInferer SET_EXPRESSION_TYPE = new IFunctionTypeInferer() {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             fd.setImmutableStates(context.getType(expr));
+        }
+    };
+
+    public static final IFunctionTypeInferer SET_STRING_OFFSET = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) {
+            fd.setImmutableStates(compilerProps.getStringOffset());
         }
     };
 
     public static final class CastTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
             IAType rt = TypeCastUtils.getRequiredType(funcExpr);
             IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
@@ -72,8 +81,8 @@
 
     public static final class DeepEqualityTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
             IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
@@ -83,8 +92,8 @@
 
     public static final class FieldAccessByIndexTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             switch (t.getTypeTag()) {
@@ -112,8 +121,8 @@
 
     public static final class FieldAccessNestedTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             AOrderedList fieldPath =
@@ -141,8 +150,8 @@
 
     public static final class GetRecordFieldsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             ATypeTag typeTag = t.getTypeTag();
@@ -158,8 +167,8 @@
 
     public static final class GetRecordFieldValueTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             ATypeTag typeTag = t.getTypeTag();
@@ -175,8 +184,8 @@
 
     public static final class OpenRecordConstructorTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             ARecordType rt = (ARecordType) context.getType(expr);
             fd.setImmutableStates(rt, computeOpenFields((AbstractFunctionCallExpression) expr, rt));
         }
@@ -204,8 +213,8 @@
 
     public static final class RecordAddFieldsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType outType = (IAType) context.getType(expr);
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
@@ -223,8 +232,8 @@
 
     public static final class RecordMergeTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType outType = (IAType) context.getType(expr);
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
@@ -235,8 +244,8 @@
 
     public static final class RecordPairsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             ATypeTag typeTag = t.getTypeTag();
@@ -252,8 +261,8 @@
 
     public static final class RecordRemoveFieldsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType outType = (IAType) context.getType(expr);
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

Integration Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/5541/ : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3260/ (10/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Till Westmann, Jenkins,

I'd like you to reexamine a change.  Please visit

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

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................

[ASTERIXDB-2272][FUN] Count character positions from 0

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

Details:
- Count character positions in strings, binary from 0 instead of 1
- Affects following built-in functions:
  position(), regexp_position(), substr()/subtring(),
  sub_binary(), find-binary()
- Introduce new configuration property “compiler.stringoffset”
  for backward compatibility
  compiler.stringoffset = 0 - count from 0 (new default)
  compiler.stringoffset = 1 - count from 1 (backwards-compatible setting)

Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
A asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
A asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
M asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
A asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
M asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
M asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
109 files changed, 872 insertions(+), 247 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/625/ (10/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2645/ : FAILURE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/6568/ (7/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1: Contrib-2

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/2582/ (6/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


[ASTERIXDB-2272][FUN] Count character positions from 0

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

Details:
- Count character positions in strings, binary from 0 instead of 1
- Affects following built-in functions:
  position(), regexp_position(), substr()/subtring(),
  sub_binary(), find-binary()
- Introduce new configuration property “compiler.stringoffset”
  for backward compatibility
  compiler.stringoffset = 0 - count from 0 (new default)
  compiler.stringoffset = 1 - count from 1 (backwards-compatible setting)

Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2342
Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
Tested-by: Dmitry Lychagin <dm...@couchbase.com>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
A asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
A asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
M asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
A asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
M asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
M asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
109 files changed, 872 insertions(+), 247 deletions(-)

Approvals:
  Dmitry Lychagin: Looks good to me, approved; Verified

Objections:
  Anon. E. Moose #1000171: Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 812f3c9..02d79c4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -20,6 +20,8 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.functions.FunctionDescriptorTag;
 import org.apache.asterix.external.library.ExternalFunctionDescriptorProvider;
@@ -64,7 +66,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = resolveFunction(expr, env);
+        IFunctionDescriptor fd = resolveFunction(expr, env, context);
         switch (fd.getFunctionDescriptorTag()) {
             case SERIALAGGREGATE:
                 return null;
@@ -82,7 +84,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return resolveFunction(expr, env).createRunningAggregateEvaluatorFactory(args);
+        return resolveFunction(expr, env, context).createRunningAggregateEvaluatorFactory(args);
     }
 
     @Override
@@ -90,7 +92,7 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        return resolveFunction(expr, env).createUnnestingEvaluatorFactory(args);
+        return resolveFunction(expr, env, context).createUnnestingEvaluatorFactory(args);
     }
 
     @Override
@@ -138,7 +140,7 @@
         IFunctionDescriptor fd = expr.getFunctionInfo() instanceof IExternalFunctionInfo
                 ? ExternalFunctionDescriptorProvider.getExternalFunctionDescriptor(
                         (IExternalFunctionInfo) expr.getFunctionInfo(), (ICcApplicationContext) context.getAppContext())
-                : resolveFunction(expr, env);
+                : resolveFunction(expr, env, context);
         return fd.createEvaluatorFactory(args);
     }
 
@@ -166,14 +168,14 @@
             AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
             JobGenContext context) throws AlgebricksException {
         IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd = resolveFunction(expr, env);
+        IFunctionDescriptor fd = resolveFunction(expr, env, context);
 
         switch (fd.getFunctionDescriptorTag()) {
             case AGGREGATE: {
                 if (BuiltinFunctions.isAggregateFunctionSerializable(fd.getIdentifier())) {
                     AggregateFunctionCallExpression serialAggExpr = BuiltinFunctions
                             .makeSerializableAggregateFunctionExpression(fd.getIdentifier(), expr.getArguments());
-                    IFunctionDescriptor afdd = resolveFunction(serialAggExpr, env);
+                    IFunctionDescriptor afdd = resolveFunction(serialAggExpr, env, context);
                     return afdd.createSerializableAggregateEvaluatorFactory(args);
                 } else {
                     throw new AlgebricksException(
@@ -192,13 +194,14 @@
         }
     }
 
-    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env)
-            throws AlgebricksException {
+    private IFunctionDescriptor resolveFunction(ILogicalExpression expr, IVariableTypeEnvironment env,
+            JobGenContext context) throws AlgebricksException {
         FunctionIdentifier fnId = ((AbstractFunctionCallExpression) expr).getFunctionIdentifier();
         IFunctionDescriptor fd = functionManager.lookupFunction(fnId);
         IFunctionTypeInferer fnTypeInfer = functionManager.lookupFunctionTypeInferer(fnId);
         if (fnTypeInfer != null) {
-            fnTypeInfer.infer(expr, fd, env);
+            CompilerProperties compilerProps = ((IApplicationContext) context.getAppContext()).getCompilerProperties();
+            fnTypeInfer.infer(expr, fd, env, compilerProps);
         }
         return fd;
     }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
new file mode 100644
index 0000000..4b7d1dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Overrides string offset to be 1 instead of 0
+ */
+@RunWith(Parameterized.class)
+public class StringOffsetTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "src/test/resources/cc-stringoffset.conf";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameterized.Parameters(name = "StringOffsetTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_stringoffset.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public StringOffsetTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf b/asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
new file mode 100644
index 0000000..c81deec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
@@ -0,0 +1,60 @@
+; 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.
+
+; Overrides string offset to be 1 instead of 0
+
+[nc/asterix_nc1]
+txn.log.dir=target/tmp/asterix_nc1/txnlog
+core.dump.dir=target/tmp/asterix_nc1/coredump
+iodevices=target/tmp/asterix_nc1/iodevice1,../asterix-server/target/tmp/asterix_nc1/iodevice2
+nc.api.port=19004
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5006
+
+[nc/asterix_nc2]
+ncservice.port=9091
+txn.log.dir=target/tmp/asterix_nc2/txnlog
+core.dump.dir=target/tmp/asterix_nc2/coredump
+iodevices=target/tmp/asterix_nc2/iodevice1,../asterix-server/target/tmp/asterix_nc2/iodevice2
+nc.api.port=19005
+#jvm.args=-agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=5007
+
+[nc]
+address=127.0.0.1
+command=asterixnc
+app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
+jvm.args=-Xmx4096m -Dnode.Resolver="org.apache.asterix.external.util.IdentitiyResolverFactory"
+storage.buffercache.pagesize=32KB
+storage.buffercache.size=48MB
+storage.memorycomponent.numpages=16
+storage.memorycomponent.globalbudget=512MB
+
+[cc]
+address = 127.0.0.1
+app.class=org.apache.asterix.hyracks.bootstrap.CCApplication
+heartbeat.period=2000
+heartbeat.max.misses=25
+
+[common]
+log.level = INFO
+compiler.framesize=32KB
+compiler.sortmemory=320KB
+compiler.groupmemory=160KB
+compiler.joinmemory=256KB
+compiler.stringoffset=1
+messaging.frame.size=4096
+messaging.frame.count=512
+
diff --git a/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql b/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
index 31087a8..646a2d4 100644
--- a/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
+++ b/asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
@@ -18,5 +18,5 @@
  */
 for $c in recordset(['cid', 'int32', 'name', 'string', 'age', 'int32'], 'osfiles', ['asterix_nc1', 'data/spj01/cust1.adm'], ['asterix_nc2', 'data/spj01/cust2.adm'])
 where some $c2 in [ $c ]
-      satisfies substring($c2.name, 1, 1) = "J"
+      satisfies substring($c2.name, 0, 1) = "J"
 return $c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
index 789cbe0..86e5d4c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
@@ -72,12 +72,12 @@
 
 create dataset Orders(OrderType)
   primary key o_orderkey;
-create dataset Customer(CustomerType) 
+create dataset Customer(CustomerType)
   primary key c_custkey;
 
 declare function q22_customer_tmp() {
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $phone_substr = '13'
     or $phone_substr = '31'
     or $phone_substr = '23'
@@ -94,7 +94,7 @@
 
 let $avg := avg(
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $c.c_acctbal > 0.00
     and ($phone_substr = '13'
     or $phone_substr = '31'
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
index 74573b9..2bbc134 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
@@ -55,7 +55,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
index 0dfdea0..7c8f918 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
@@ -55,7 +55,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
index 7b49736..22bff52 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
@@ -82,14 +82,14 @@
 (
     select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
     from  Customer as c
-    with  phone_substr as tpch.substring(c.c_phone,1,2)
+    with  phone_substr as tpch.substring(c.c_phone,0,2)
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
 with  avg as tpch.coll_avg((
       select element c.c_acctbal
       from  Customer as c
-      with  phone_substr as tpch.substring(c.c_phone,1,2)
+      with  phone_substr as tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
 select element {'cntrycode':cntrycode,'numcust':tpch.coll_count(g),'totacctbal':tpch.coll_sum((
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
index 11df55d..62f92a7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
@@ -83,7 +83,7 @@
           Variable [ Name=$c ]
           Field=c_phone
         ]
-        LiteralExpr [LONG] [1]
+        LiteralExpr [LONG] [0]
         LiteralExpr [LONG] [2]
       ]
     Where
@@ -156,7 +156,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       Where
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
index 58f6a96..6e64f16 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
@@ -19,13 +19,13 @@
 use dataverse test;
 
 let $x := hex("aabbccddaa")
-let $r1 := find-binary($x, hex('')) = 1
-let $r2 := find-binary($x, hex('aa')) = 1
-let $r3 := find-binary($x, hex('aa'), 1) = 5
+let $r1 := find-binary($x, hex('')) = 0
+let $r2 := find-binary($x, hex('aa')) = 0
+let $r3 := find-binary($x, hex('aa'), 1) = 4
 let $r4 := find-binary($x, hex('aabb'), 0) = find-binary($x, hex('aabb'))
-let $r5 := find-binary($x, hex('11')) = 0
-let $r6 := find-binary($x, hex('ccddaa')) = 3
-let $r7 := find-binary($x, hex('ccddaabb')) = 0
+let $r5 := find-binary($x, hex('11')) = -1
+let $r6 := find-binary($x, hex('ccddaa')) = 2
+let $r7 := find-binary($x, hex('ccddaabb')) = -1
 
 let $r8 := find-binary($x, null)
 let $r9 := find-binary(null, null)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
index d4dce94..79980af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
@@ -24,18 +24,18 @@
 let $r2 := sub-binary(hex(''),1) = hex('')
 let $r3 := sub-binary(hex(''),-1) = hex('')
 
-let $r4 := sub-binary($x, 1, binary-length($x)) = $x
-let $r5 := sub-binary($x, 2, 1) = hex('bb')
-let $r6 := sub-binary($x, 2) = hex('bbccdd')
-let $r7 := sub-binary($x, 5, 0) = hex('')
-let $r8 := sub-binary($x, 4, 1) = hex('dd')
-let $r9 := sub-binary($x, 2, 2) = hex('bbcc') 
+let $r4 := sub-binary($x, 0, binary-length($x)) = $x
+let $r5 := sub-binary($x, 1, 1) = hex('bb')
+let $r6 := sub-binary($x, 1) = hex('bbccdd')
+let $r7 := sub-binary($x, 4, 0) = hex('')
+let $r8 := sub-binary($x, 3, 1) = hex('dd')
+let $r9 := sub-binary($x, 1, 2) = hex('bbcc')
 
 let $r10 := sub-binary($x, 0) = $x
 let $r11 := sub-binary($x, -1) = $x
-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('')
+let $r12 := sub-binary($x, 0, 256) = $x
+let $r13 := sub-binary($x, 1, 256) = hex('bbccdd')
+let $r14 := sub-binary($x, 1, -1) = hex('')
 
 return  [ $r1 ,$r2 ,$r3 ,$r4 ,$r5 , $r6 , $r7 , $r8 , $r9 , $r10 , $r11 , $r12 , $r13 , $r14 ]
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
index 0c20cec..bc9e7f7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
@@ -24,13 +24,13 @@
  */
 
 let $str1:="Hello World"
-let $str2:=substring($str1,10)
+let $str2:=substring($str1,9)
 
 let $str3:="This is a test string"
-let $str4:=substring($str3,21)
+let $str4:=substring($str3,20)
 
 let $str5:="This is a test string"
-let $str6:=substring($str5,22)
+let $str6:=substring($str5,21)
 
 let $str7:="This is a test string"
 let $str8:=substring($str7,0)
@@ -40,7 +40,7 @@
 
 let $str11:="This is a test string"
 let $str12:="This is a another test string"
-let $str13:=substring(string-concat([$str11,$str12]),21)
+let $str13:=substring(string-concat([$str11,$str12]),20)
 
-let $str14:=substring("UC Irvine",string-length("UC Irvine")/2)
+let $str14:=substring("UC Irvine",string-length("UC Irvine")/2 - 1)
 return { "str2":$str2,"str4":$str4,"str6":$str6,"str8":$str8,"str10":$str10,"str13":$str13,"str14":$str14}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
index ee98756..1c31ea0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
@@ -23,13 +23,13 @@
  * Date           : 18th April 2012
  */
 
-for $a in [ substring("hello world", 7, 5),
-substring("hello world", 1, 11),
-substring("hello world", 3, 9),
-substring("ABCD", 3, 2),
-substring("ABCD", 1, 4),
-substring("UC Irvine", 4, string-length("UC Irvine") - 3),
-substring("UC Irvine", 1, string-length("UC Irvine")),
-substring(substring("UC Irvine", 4), 1, string-length("Irvine"))
+for $a in [ substring("hello world", 6, 5),
+substring("hello world", 0, 11),
+substring("hello world", 2, 9),
+substring("ABCD", 2, 2),
+substring("ABCD", 0, 4),
+substring("UC Irvine", 3, string-length("UC Irvine") - 3),
+substring("UC Irvine", 0, string-length("UC Irvine")),
+substring(substring("UC Irvine", 3), 0, string-length("Irvine"))
 ]
 return $a
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
index cd6ab0a..d500ac5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
@@ -29,4 +29,4 @@
 
 for $a in dataset('testdst')
 order by $a.name
-return substring($a.name, 4, string-length($a.name) - 3);
+return substring($a.name, 3, string-length($a.name) - 3);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
index 0f41b6e..078fd0c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
@@ -28,4 +28,4 @@
 
 for $a in dataset('testdst')
 order by $a.name
-return substring($a.name,4);
+return substring($a.name,3);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
index 57ddd9a..26337e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
@@ -18,5 +18,5 @@
  */
 use dataverse test;
 
-let $c1 := substring("HEllow",2)
+let $c1 := substring("HEllow",1)
 return {"result1": $c1}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
index e230f91..385db08 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
@@ -18,5 +18,5 @@
  */
 use dataverse test;
 
-let $c1 := substring("HEllow",10)
+let $c1 := substring("HEllow",9)
 return {"result1": $c1}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
index dc94226..752ec61 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
@@ -19,4 +19,4 @@
 use dataverse test;
 
 for $x in ["foobar"]
-return substring($x, 2, 3)
+return substring($x, 1, 3)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
index e8cf4e8..ae5f99a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
@@ -23,7 +23,7 @@
   select {
     "c_acctbal": $c.c_acctbal,
     "c_custkey": $c.c_custkey,
-    "cntrycode": substring($c.c_phone, 1, 2)
+    "cntrycode": substring($c.c_phone, 0, 2)
   }
 }
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
index 0c7f979..365e84a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
@@ -23,7 +23,7 @@
   return {
     "c_acctbal": $c.c_acctbal,
     "c_custkey": $c.c_custkey,
-    "cntrycode": substring($c.c_phone, 1, 2)
+    "cntrycode": substring($c.c_phone, 0, 2)
   }
 }
 
@@ -37,7 +37,7 @@
 group by $cntrycode := $ct.cntrycode with $ct
 order by $cntrycode
 return {
-  "cntrycode": $cntrycode, 
-  "numcust": count($ct), 
+  "cntrycode": $cntrycode,
+  "numcust": count($ct),
   "totacctbal": sum(for $i in $ct return $i.c_acctbal)
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
index 4aa0e13..24db233 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
@@ -27,7 +27,7 @@
 
 declare function q22_customer_tmp() {
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $phone_substr = '13'
     or $phone_substr = '31'
     or $phone_substr = '23'
@@ -44,7 +44,7 @@
 
 let $avg := avg(
   for $c in dataset('Customer')
-  let $phone_substr := substring($c.c_phone, 1, 2)
+  let $phone_substr := substring($c.c_phone, 0, 2)
   where $c.c_acctbal > 0.00
     and ($phone_substr = '13'
     or $phone_substr = '31'
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
index 2b596c6..1660974 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
@@ -19,5 +19,18 @@
 
 use test;
 
-
-[(test.`find-binary`(test.hex('aabbccddaa'),test.hex('')) = 1),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa')) = 1),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa'),1) = 5),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'),0) = test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'))),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('11')) = 0),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaa')) = 3),(test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaabb')) = 0),test.`find-binary`(test.hex('aabbccddaa'),null),test.`find-binary`(null,null),test.`find-binary`(null,test.hex('aabbccddaa'))];
+[
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('')) = 0),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa')) = 0),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('aa'),1) = 4),
+  (
+    test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'),-1) =
+    test.`find-binary`(test.hex('aabbccddaa'),test.hex('aabb'))
+  ),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('11')) = -1),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaa')) = 2),
+  (test.`find-binary`(test.hex('aabbccddaa'),test.hex('ccddaabb')) = -1),
+  test.`find-binary`(test.hex('aabbccddaa'),null),
+  test.`find-binary`(null,null),
+  test.`find-binary`(null,test.hex('aabbccddaa'))
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
index a1141b5..99ef6fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
@@ -20,4 +20,19 @@
 use test;
 
 
-[(test.`sub-binary`(test.hex(''),0) = test.hex('')),(test.`sub-binary`(test.hex(''),1) = test.hex('')),(test.`sub-binary`(test.hex(''),-1) = test.hex('')),(test.`sub-binary`(test.hex('aabbccdd'),1,test.`binary-length`(test.hex('aabbccdd'))) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),2,1) = test.hex('bb')),(test.`sub-binary`(test.hex('aabbccdd'),2) = test.hex('bbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),5,0) = test.hex('')),(test.`sub-binary`(test.hex('aabbccdd'),4,1) = test.hex('dd')),(test.`sub-binary`(test.hex('aabbccdd'),2,2) = test.hex('bbcc')),(test.`sub-binary`(test.hex('aabbccdd'),0) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),-1) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),1,256) = test.hex('aabbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),2,256) = test.hex('bbccdd')),(test.`sub-binary`(test.hex('aabbccdd'),2,-1) = test.hex(''))];
+[
+    (test.`sub-binary`(test.hex(''),0) = test.hex('')),
+    (test.`sub-binary`(test.hex(''),1) = test.hex('')),
+    (test.`sub-binary`(test.hex(''),-1) = test.hex('')),
+    (test.`sub-binary`(test.hex('aabbccdd'),0,test.`binary-length`(test.hex('aabbccdd'))) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,1) = test.hex('bb')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1) = test.hex('bbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),4,0) = test.hex('')),
+    (test.`sub-binary`(test.hex('aabbccdd'),3,1) = test.hex('dd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,2) = test.hex('bbcc')),
+    (test.`sub-binary`(test.hex('aabbccdd'),0) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),-1) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),0,256) = test.hex('aabbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,256) = test.hex('bbccdd')),
+    (test.`sub-binary`(test.hex('aabbccdd'),1,-1) = test.hex(''))
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
index a4166e5..25017f8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
@@ -21,5 +21,5 @@
 
 EXPLAIN
 FROM Employee e
-GROUP BY substr(e.dept.department_id, 1)
-SELECT substr(e.dept.department_id, 1) as deptId, SUM(e.salary) AS star_cost;
+GROUP BY substr(e.dept.department_id, 0)
+SELECT substr(e.dept.department_id, 0) as deptId, SUM(e.salary) AS star_cost;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
index e170a6e..05179af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
@@ -21,5 +21,5 @@
 
 EXPLAIN
 FROM Employee e
-GROUP BY substr(e.department_id, 1)
-SELECT substr(e.department_id, 1) as deptId, SUM(e.salary) AS star_cost;
+GROUP BY substr(e.department_id, 0)
+SELECT substr(e.department_id, 0) as deptId, SUM(e.salary) AS star_cost;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
index 5c6cc38..8e8ce59 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
@@ -22,6 +22,6 @@
 FROM Employee e
     JOIN Incentive i ON e.job_category = i.job_category
     JOIN SuperStars s ON e.id = s.id
-GROUP BY substr(e.department_id, 1)
-SELECT substr(e.department_id, 1), SUM(e.salary + i.bonus);
+GROUP BY substr(e.department_id, 0)
+SELECT substr(e.department_id, 0), SUM(e.salary + i.bonus);
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
index c2d046d..eb6fa1d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
@@ -22,6 +22,6 @@
 FROM Employee e
     JOIN Incentive i ON e.job_category = i.job_category
     JOIN SuperStars s ON e.id = s.id
-GROUP BY substr(e.department_id, 1)
-SELECT substr(e.department_id, 1) as deptId, SUM(e.salary + i.bonus) AS star_cost;
+GROUP BY substr(e.department_id, 0)
+SELECT substr(e.department_id, 0) as deptId, SUM(e.salary + i.bonus) AS star_cost;
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
index d6bb472..10e525f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
@@ -18,24 +18,24 @@
  */
 
 {
-  'a': SUBSTR(null, 1),
+  'a': SUBSTR(null, 0),
   'b': SUBSTR('abc', null),
   'c': SUBSTR(null, null),
-  'd': SUBSTR(null, 1, 2),
+  'd': SUBSTR(null, 0, 2),
   'e': substr('abc', null, 2),
-  'f': SUBSTR('abc', 1, null),
+  'f': SUBSTR('abc', 0, null),
   'g': SUBSTR(null, null, 2),
-  'h': substr(null, 1, null),
+  'h': substr(null, 0, null),
   'i': SUBSTR(null, null, null),
   'j': SUBSTR('abc', null, null),
-  'k': SUBSTR(missing, 1),
+  'k': SUBSTR(missing, 0),
   'l': SUBSTR('abc', missing),
   'm': SUBSTR(missing, missing),
-  'o': SUBSTR(missing, 1, 2),
+  'o': SUBSTR(missing, 0, 2),
   'p': substr('abc', missing, 2),
-  'q': SUBSTR('abc', 1, missing),
+  'q': SUBSTR('abc', 0, missing),
   'r': SUBSTR(missing, missing, 2),
-  's': substr(missing, 1, missing),
+  's': substr(missing, 0, missing),
   't': SUBSTR(missing, missing, missing),
   'u': SUBSTR('abc', missing, missing),
   'v': SUBSTR(null, missing, missing),
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
index 3fa0247..b9c79d0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
@@ -23,4 +23,4 @@
  * Date           : 18th April 2012
  */
 
-{'str2':substring('Hello World',10),'str4':substring('This is a test string',21),'str6':substring('This is a test string',22),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-1),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),21),'str14':substring('UC Irvine',(`string-length`('UC Irvine') / 2))};
+{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-1),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') / 2 - 1))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
index a088506..860173e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
@@ -24,5 +24,5 @@
  */
 
 select element a
-from  [substring('hello world',7,5),substring('hello world',1,11),substring('hello world',3,9),substring('ABCD',3,2),substring('ABCD',1,4),substring('UC Irvine',4,(`string-length`('UC Irvine') - 3)),substring('UC Irvine',1,`string-length`('UC Irvine')),substring(substring('UC Irvine',4),1,`string-length`('Irvine'))] as a
+from  [substring('hello world',6,5),substring('hello world',0,11),substring('hello world',2,9),substring('ABCD',2,2),substring('ABCD',0,4),substring('UC Irvine',3,(`string-length`('UC Irvine') - 3)),substring('UC Irvine',0,`string-length`('UC Irvine')),substring(substring('UC Irvine',3),0,`string-length`('Irvine'))] as a
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
index 2bfa797..9926926 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
@@ -26,7 +26,7 @@
 use test;
 
 
-select element test.substring(a.name,4,(test.`string-length`(a.name) - 3))
+select element test.substring(a.name,3,(test.`string-length`(a.name) - 3))
 from  testdst as a
 order by a.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
index 991a911..dfd8ebb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
@@ -25,7 +25,7 @@
 use test;
 
 
-select element test.substring(a.name,4)
+select element test.substring(a.name,3)
 from  testdst as a
 order by a.name
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
index 27ff9ae..fa3b8af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
@@ -20,4 +20,4 @@
 use test;
 
 
-{'result1':test.substring('HEllow',2)};
+{'result1':test.substring('HEllow',1)};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
index eee0126..9f1fbc6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
@@ -20,4 +20,4 @@
 use test;
 
 
-{'result1':test.substring('HEllow',10)};
+{'result1':test.substring('HEllow',9)};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
index 40a6960..1ad0a0f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
@@ -20,6 +20,6 @@
 use test;
 
 
-select element test.substring(x,2,3)
+select element test.substring(x,1,3)
 from  ['foobar'] as x
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
new file mode 100644
index 0000000..073220f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),-1),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),0),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),1),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('aabb'),2),
+    find_binary(hex('aabbccddeeffaabbccddeeff'),hex('1122'))
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp
new file mode 100644
index 0000000..7ea703e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    find_binary(hex('aabbccddeeff'),hex('aabb')),
+    find_binary(hex('aabbccddeeff'),hex('ccdd')),
+    find_binary(hex('aabbccddeeff'),hex('1122'))
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp
new file mode 100644
index 0000000..8f321c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    position('abcdefg','ab'),
+    position('abcdefg','de'),
+    position('abc', 'de')
+]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp
new file mode 100644
index 0000000..7a27b2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+  REGEXP_POSITION('mnop','.'),
+  REGEXP_POSITION('mnop','p'),
+  REGEXP_POSITION('abcdefABCDEF','/d')
+]
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp
new file mode 100644
index 0000000..ac47eb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+  REGEXP_POSITION('mnop','.','x'),
+  REGEXP_POSITION('mnop','p','x'),
+  REGEXP_POSITION('abcdefABCDEF','/d','x')
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
new file mode 100644
index 0000000..6989cf7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    print_binary(sub_binary(hex('aabbccddeeff'),-1,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),0,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),1,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),2,2), "hex"),
+    print_binary(sub_binary(hex('aabbccddeeff'),3,2), "hex")
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
new file mode 100644
index 0000000..075f803
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    print_binary(sub_binary(hex('aabbccddeeff'),-1), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),0), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),1), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),2), 'hex'),
+    print_binary(sub_binary(hex('aabbccddeeff'),3), 'hex')
+]
+;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
new file mode 100644
index 0000000..1c35e7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    substring('abcdefg',-1,2),
+    substring('abcdefg',0,2),
+    substring('abcdefg',1,2),
+    substring('abcdefg',2,2),
+    substring('abcdefg',3,2)
+]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
new file mode 100644
index 0000000..65d10b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+
+/* String offset is set to 1 */
+
+select value [
+    substring('abcdefg',-1),
+    substring('abcdefg',0),
+    substring('abcdefg',1),
+    substring('abcdefg',2),
+    substring('abcdefg',3)
+]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
index e7328f3..cdaad5e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
@@ -21,7 +21,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 ),
 avg AS (
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
index c47aad3..53d2a68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
@@ -21,7 +21,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
index 66e99ab..2f46ea7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
@@ -23,7 +23,7 @@
 
 SELECT  user.name AS uname,
         user.alias AS alias,
-        substring(message.message, 30) AS msg
+        substring(message.message, 29) AS msg
 FROM FacebookMessages AS message,
      FacebookUsers AS user
 WHERE message.`author-id` = user.id and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
index 7f84895..a8f43a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
@@ -21,7 +21,7 @@
 
 SELECT  user.name AS uname,
         user.alias AS alias,
-        substring(message.message, 30) AS msg
+        substring(message.message, 29) AS msg
 FROM FacebookMessages AS message,
      FacebookUsers AS user
 WHERE message.`author-id` = user.id and
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
index 484757a..7152cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
@@ -28,7 +28,7 @@
     ,date_dim d
 WHERE cs.cs_bill_customer_sk = c.c_customer_sk
 AND c.c_current_addr_sk = ca.ca_address_sk
-AND ( substr(ca.ca_zip,1,5) in ['85669', '86197','88274','83405','86475',
+AND ( substr(ca.ca_zip,0,5) in ['85669', '86197','88274','83405','86475',
                              '85392', '85460', '80348', '81792']
 OR ca.ca_state in ['CA','WA','GA']
 OR cs.cs_sales_price > 500)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
index 5e3b339..52c298e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
@@ -30,7 +30,7 @@
 AND d.d_year=1999
 AND ss.ss_customer_sk = c.c_customer_sk
 AND c.c_current_addr_sk = ca.ca_address_sk
-AND substr(ca.ca_zip,1,5) != substr(s.s_zip,1,5)
+AND substr(ca.ca_zip,0,5) != substr(s.s_zip,0,5)
 AND ss.ss_store_sk = s.s_store_sk
 GROUP BY i.i_brand
         ,i.i_brand_id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
index d2a0d85..2d8fdc9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
@@ -24,7 +24,7 @@
  WHERE ws.ws_bill_customer_sk = c.c_customer_sk
   AND c.c_current_addr_sk = ca.ca_address_sk
   AND ws.ws_item_sk = i2.i_item_sk
-  AND ( substr(ca.ca_zip,1,5) IN ['85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792']
+  AND ( substr(ca.ca_zip,0,5) IN ['85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792']
         OR
         i2.i_item_id IN (SELECT VALUE i1.i_item_id
                          FROM item i1
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
index b328964..6f8eb1e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
@@ -21,7 +21,7 @@
 USE tpcds;
 
 SELECT
-   SUBSTR(w.w_warehouse_name,1,20)
+   SUBSTR(w.w_warehouse_name,0,20)
   ,sm.sm_type
   ,web.web_name
   ,SUM(CASE WHEN (ws.ws_ship_date_sk - ws.ws_sold_date_sk) <= 30 THEN 1 ELSE 0 END)  AS c30_days
@@ -45,10 +45,10 @@
 AND ws.ws_ship_mode_sk   = sm.sm_ship_mode_sk
 AND ws.ws_web_site_sk    = web.web_site_sk
 GROUP BY
-   SUBSTR(w.w_warehouse_name,1,20)
+   SUBSTR(w.w_warehouse_name,0,20)
   ,sm.sm_type
   ,web.web_name
-ORDER BY SUBSTR(w.w_warehouse_name,1,20)
+ORDER BY SUBSTR(w.w_warehouse_name,0,20)
         ,sm_type
        ,web_name
 LIMIT 100;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
index 0bd5649..ea27bdf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
@@ -21,7 +21,7 @@
 USE tpcds;
 
 SELECT
-  c.c_last_name,c.c_first_name,substr(ms.s_city,1,30),ms.ss_ticket_number,ms.amt,ms.profit
+  c.c_last_name,c.c_first_name,substr(ms.s_city,0,30),ms.ss_ticket_number,ms.amt,ms.profit
   FROM
    (SELECT ss_ticket_number
           ,ss_customer_sk
@@ -38,5 +38,5 @@
     AND store.s_number_employees >= 200 AND store.s_number_employees <= 295
     group by store_sales.ss_ticket_number,store_sales.ss_customer_sk,store_sales.ss_addr_sk,store.s_city) ms,customer c
     WHERE ms.ss_customer_sk = c.c_customer_sk
- ORDER BY c.c_last_name,c.c_first_name,SUBSTR(ms.s_city,1,30), ms.profit
+ ORDER BY c.c_last_name,c.c_first_name,SUBSTR(ms.s_city,0,30), ms.profit
  LIMIT 100;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
index 8ab3f25..7951333 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
@@ -20,7 +20,7 @@
 
 use tpcds;
 
-select  substr(r_reason_desc,1,20)
+select  substr(r_reason_desc,0,20)
        ,avg(ws.ws_quantity)
        ,avg(wr.wr_refunded_cash)
        ,avg(wr.wr_fee)
@@ -96,7 +96,7 @@
     )
    )
 group by r.r_reason_desc
-order by substr(r_reason_desc,1,20)
+order by substr(r_reason_desc,0,20)
         ,avg(ws.ws_quantity)
         ,avg(wr.wr_refunded_cash)
         ,avg(wr.wr_fee)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index bdd7323..d33a095 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,1,2)}
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,0,2)}
     from  Customer as c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index f873dc6..a3a4473 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -21,7 +21,7 @@
 
 WITH q22_customer_tmp AS
 (
-    SELECT c_acctbal, c_custkey, substring(c_phone,1,2) AS cntrycode
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
     FROM  Customer
 )
 ,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
index 4a23a50..c6afe18 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
@@ -30,7 +30,7 @@
 (
     SELECT c.c_acctbal, c.c_custkey, phone_substr AS cntrycode
     FROM  Customer c
-    LET  phone_substr = substring(c_phone,1,2)
+    LET  phone_substr = substring(c_phone,0,2)
     WHERE phone_substr = '13' OR phone_substr = '31' OR phone_substr = '23' OR phone_substr = '29'
           OR phone_substr = '30' OR phone_substr = '18' OR phone_substr = '17'
 )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index 109e5a9..206033a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    SELECT c.c_acctbal AS c_acctbal, c.c_custkey AS c_custkey, substring(c.c_phone,1,2) AS cntrycode
+    SELECT c.c_acctbal AS c_acctbal, c.c_custkey AS c_custkey, substring(c.c_phone,0,2) AS cntrycode
     FROM  Customer AS c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
index 3c99324..531e17f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
@@ -30,7 +30,7 @@
 (
     SELECT c.c_acctbal AS c_acctbal, c.c_custkey AS c_custkey, phone_substr AS cntrycode
     FROM  Customer AS c
-    WITH  phone_substr AS substring(c.c_phone,1,2)
+    WITH  phone_substr AS substring(c.c_phone,0,2)
     WHERE phone_substr = '13' OR phone_substr = '31' OR phone_substr = '23' OR phone_substr = '29'
           OR phone_substr = '30' OR phone_substr = '18' OR phone_substr = '17'
 )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index bdd7323..d33a095 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,1,2)}
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,0,2)}
     from  Customer as c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
index dc54947..4cd5d54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
@@ -30,14 +30,14 @@
 (
     select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
     from  Customer as c
-    let  phone_substr = tpch.substring(c.c_phone,1,2)
+    let  phone_substr = tpch.substring(c.c_phone,0,2)
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
 with  avg as tpch.coll_avg((
       select element c.c_acctbal
       from  Customer as c
-      let  phone_substr = tpch.substring(c.c_phone,1,2)
+      let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
 select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
index bdd7323..d33a095 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
@@ -22,7 +22,7 @@
 
 declare function q22_customer_tmp() {
 (
-    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,1,2)}
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':SUBSTR(c.c_phone,0,2)}
     from  Customer as c
 )
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
index dc54947..4cd5d54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
@@ -30,14 +30,14 @@
 (
     select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
     from  Customer as c
-    let  phone_substr = tpch.substring(c.c_phone,1,2)
+    let  phone_substr = tpch.substring(c.c_phone,0,2)
     where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
 )
 };
 with  avg as tpch.coll_avg((
       select element c.c_acctbal
       from  Customer as c
-      let  phone_substr = tpch.substring(c.c_phone,1,2)
+      let  phone_substr = tpch.substring(c.c_phone,0,2)
       where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
   ))
 select element {'cntrycode':cntrycode,'numcust':tpch.count(ct),'totacctbal':tpch.coll_sum((
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
index 5f25e64..98cedcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
@@ -28,7 +28,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   project ([$$34, $$37])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$37] <- [substring($$41.getField("department_id"), 1)]
+                    assign [$$37] <- [substring($$41.getField("department_id"), 0)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$34, $$41])
                       -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
index 92df490..e2e1c91 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
@@ -28,7 +28,7 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   project ([$$33, $$36])
                   -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$36, $$33] <- [substring($$e.getField(1), 1), $$e.getField(2)]
+                    assign [$$36, $$33] <- [substring($$e.getField(1), 0), $$e.getField(2)]
                     -- ASSIGN  |PARTITIONED|
                       project ([$$e])
                       -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
index 5c54d04..76e3e97 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
@@ -1 +1 @@
-[ 3, -1 ]
+[ 2, -1 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
index 83ee615..428917f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
@@ -1,15 +1,15 @@
-1
+0
 -1
-1
-1
-1
+0
+0
+0
 -1
-1
+0
 -1
 -1
-1
-1
-1
-1
+0
+0
+0
+0
 -1
-1
+0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
index f962896..fa99af7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
@@ -1 +1 @@
-{ "result1": 1 }
+{ "result1": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
new file mode 100644
index 0000000..0fae953
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
@@ -0,0 +1 @@
+[ 1, 1, 1, 7, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
new file mode 100644
index 0000000..299ae43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
@@ -0,0 +1 @@
+[ 1, 3, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
new file mode 100644
index 0000000..613f89b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
@@ -0,0 +1 @@
+[ 1, 4, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
new file mode 100644
index 0000000..613f89b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
@@ -0,0 +1 @@
+[ 1, 4, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
new file mode 100644
index 0000000..613f89b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
@@ -0,0 +1 @@
+[ 1, 4, -1 ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
new file mode 100644
index 0000000..f5e809d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
@@ -0,0 +1 @@
+[ "AABB", "AABB", "AABB", "BBCC", "CCDD" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
new file mode 100644
index 0000000..275d27a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
@@ -0,0 +1 @@
+[ "AABBCCDDEEFF", "AABBCCDDEEFF", "AABBCCDDEEFF", "BBCCDDEEFF", "CCDDEEFF" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
new file mode 100644
index 0000000..b92a705
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
@@ -0,0 +1 @@
+[ "ab", "ab", "ab", "bc", "cd" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
new file mode 100644
index 0000000..e341861
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
@@ -0,0 +1 @@
+[ "abcdefg", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
index cd2f25c..813591c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
@@ -11,7 +11,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [1]
+    LiteralExpr [LONG] [0]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@2[
@@ -23,7 +23,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [1]
+    LiteralExpr [LONG] [0]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@3[
@@ -36,7 +36,7 @@
       LiteralExpr [LONG] [1]
     ]
     =
-    LiteralExpr [LONG] [5]
+    LiteralExpr [LONG] [4]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@3[
@@ -46,7 +46,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabb]
       ]
-      LiteralExpr [LONG] [0]
+      - LiteralExpr [LONG] [1]
     ]
     =
     FunctionCall test.find-binary@2[
@@ -68,7 +68,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [0]
+    - LiteralExpr [LONG] [1]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@2[
@@ -80,7 +80,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [3]
+    LiteralExpr [LONG] [2]
   ]
   OperatorExpr [
     FunctionCall test.find-binary@2[
@@ -92,7 +92,7 @@
       ]
     ]
     =
-    LiteralExpr [LONG] [0]
+    - LiteralExpr [LONG] [1]
   ]
   FunctionCall test.find-binary@2[
     FunctionCall test.hex@1[
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
index 6ec8199..5a232ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
@@ -42,7 +42,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       FunctionCall test.binary-length@1[
         FunctionCall test.hex@1[
           LiteralExpr [STRING] [aabbccdd]
@@ -59,7 +59,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       LiteralExpr [LONG] [1]
     ]
     =
@@ -72,7 +72,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
     ]
     =
     FunctionCall test.hex@1[
@@ -84,7 +84,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [5]
+      LiteralExpr [LONG] [4]
       LiteralExpr [LONG] [0]
     ]
     =
@@ -97,7 +97,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [4]
+      LiteralExpr [LONG] [3]
       LiteralExpr [LONG] [1]
     ]
     =
@@ -110,7 +110,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       LiteralExpr [LONG] [2]
     ]
     =
@@ -147,7 +147,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       LiteralExpr [LONG] [256]
     ]
     =
@@ -160,7 +160,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       LiteralExpr [LONG] [256]
     ]
     =
@@ -173,7 +173,7 @@
       FunctionCall test.hex@1[
         LiteralExpr [STRING] [aabbccdd]
       ]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
       - LiteralExpr [LONG] [1]
     ]
     =
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
index 9e8f66d..a86d0a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
@@ -5,7 +5,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [Hello World]
-      LiteralExpr [LONG] [10]
+      LiteralExpr [LONG] [9]
     ]
   )
   (
@@ -13,7 +13,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [This is a test string]
-      LiteralExpr [LONG] [21]
+      LiteralExpr [LONG] [20]
     ]
   )
   (
@@ -21,7 +21,7 @@
     :
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [This is a test string]
-      LiteralExpr [LONG] [22]
+      LiteralExpr [LONG] [21]
     ]
   )
   (
@@ -50,7 +50,7 @@
           LiteralExpr [STRING] [This is a another test string]
         ]
       ]
-      LiteralExpr [LONG] [21]
+      LiteralExpr [LONG] [20]
     ]
   )
   (
@@ -59,11 +59,15 @@
     FunctionCall null.substring@2[
       LiteralExpr [STRING] [UC Irvine]
       OperatorExpr [
-        FunctionCall null.string-length@1[
-          LiteralExpr [STRING] [UC Irvine]
+        OperatorExpr [
+          FunctionCall null.string-length@1[
+            LiteralExpr [STRING] [UC Irvine]
+          ]
+          /
+          LiteralExpr [LONG] [2]
         ]
-        /
-        LiteralExpr [LONG] [2]
+        -
+        LiteralExpr [LONG] [1]
       ]
     ]
   )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
index ccd395d..f85b949 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
@@ -5,32 +5,32 @@
 FROM [  OrderedListConstructor [
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [hello world]
-      LiteralExpr [LONG] [7]
+      LiteralExpr [LONG] [6]
       LiteralExpr [LONG] [5]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [hello world]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       LiteralExpr [LONG] [11]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [hello world]
-      LiteralExpr [LONG] [3]
+      LiteralExpr [LONG] [2]
       LiteralExpr [LONG] [9]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [ABCD]
-      LiteralExpr [LONG] [3]
+      LiteralExpr [LONG] [2]
       LiteralExpr [LONG] [2]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [ABCD]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       LiteralExpr [LONG] [4]
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [UC Irvine]
-      LiteralExpr [LONG] [4]
+      LiteralExpr [LONG] [3]
       OperatorExpr [
         FunctionCall null.string-length@1[
           LiteralExpr [STRING] [UC Irvine]
@@ -41,7 +41,7 @@
     ]
     FunctionCall null.substring@3[
       LiteralExpr [STRING] [UC Irvine]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       FunctionCall null.string-length@1[
         LiteralExpr [STRING] [UC Irvine]
       ]
@@ -49,9 +49,9 @@
     FunctionCall null.substring@3[
       FunctionCall null.substring@2[
         LiteralExpr [STRING] [UC Irvine]
-        LiteralExpr [LONG] [4]
+        LiteralExpr [LONG] [3]
       ]
-      LiteralExpr [LONG] [1]
+      LiteralExpr [LONG] [0]
       FunctionCall null.string-length@1[
         LiteralExpr [STRING] [Irvine]
       ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
index d02dca1..685a3b4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
@@ -6,7 +6,7 @@
     Variable [ Name=$a ]
     Field=name
   ]
-  LiteralExpr [LONG] [4]
+  LiteralExpr [LONG] [3]
   OperatorExpr [
     FunctionCall test.string-length@1[
       FieldAccessor [
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
index 4f1d3f4..f27bf4d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
@@ -6,7 +6,7 @@
     Variable [ Name=$a ]
     Field=name
   ]
-  LiteralExpr [LONG] [4]
+  LiteralExpr [LONG] [3]
 ]
 ]
 FROM [  FunctionCall asterix.dataset@1[
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
index ed84aaa..c170655 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
@@ -6,7 +6,7 @@
     :
     FunctionCall test.substring@2[
       LiteralExpr [STRING] [HEllow]
-      LiteralExpr [LONG] [2]
+      LiteralExpr [LONG] [1]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
index 865e373..358d135 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
@@ -6,7 +6,7 @@
     :
     FunctionCall test.substring@2[
       LiteralExpr [STRING] [HEllow]
-      LiteralExpr [LONG] [10]
+      LiteralExpr [LONG] [9]
     ]
   )
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
index cd3b906..d24c5b8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
@@ -3,7 +3,7 @@
 SELECT ELEMENT [
 FunctionCall test.substring@3[
   Variable [ Name=$x ]
-  LiteralExpr [LONG] [2]
+  LiteralExpr [LONG] [1]
   LiteralExpr [LONG] [3]
 ]
 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
index 7425817..cdd8a53 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
@@ -16,7 +16,7 @@
     Variable [ Name=$message ]
     Field=message
   ]
-  LiteralExpr [LONG] [30]
+  LiteralExpr [LONG] [29]
 ]
 msg
 ]
@@ -78,7 +78,7 @@
       Variable [ Name=$message ]
       Field=message
     ]
-    LiteralExpr [LONG] [30]
+    LiteralExpr [LONG] [29]
   ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
index 7425817..cdd8a53 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
@@ -16,7 +16,7 @@
     Variable [ Name=$message ]
     Field=message
   ]
-  LiteralExpr [LONG] [30]
+  LiteralExpr [LONG] [29]
 ]
 msg
 ]
@@ -78,7 +78,7 @@
       Variable [ Name=$message ]
       Field=message
     ]
-    LiteralExpr [LONG] [30]
+    LiteralExpr [LONG] [29]
   ]
   ASC
 
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
index 9b94363..0ad2c82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
@@ -27,7 +27,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
index 9b94363..0ad2c82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
@@ -27,7 +27,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       )
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
index d9f60e3..928d5e2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
@@ -36,7 +36,7 @@
           Variable [ Name=$c ]
           Field=c_phone
         ]
-        LiteralExpr [LONG] [1]
+        LiteralExpr [LONG] [0]
         LiteralExpr [LONG] [2]
       ]
     Where
@@ -109,7 +109,7 @@
             Variable [ Name=$c ]
             Field=c_phone
           ]
-          LiteralExpr [LONG] [1]
+          LiteralExpr [LONG] [0]
           LiteralExpr [LONG] [2]
         ]
       Where
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
new file mode 100644
index 0000000..f7da162
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
@@ -0,0 +1,67 @@
+<!--
+ ! 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.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+  <test-group name="stringoffset">
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="position">
+        <output-dir compare="Text">position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="regexp_position">
+        <output-dir compare="Text">regexp_position</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="regexp_position_with_flag">
+        <output-dir compare="Text">regexp_position_with_flag</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="substring">
+        <output-dir compare="Text">substring</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="substring2">
+        <output-dir compare="Text">substring2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="subbinary">
+        <output-dir compare="Text">subbinary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="subbinary2">
+        <output-dir compare="Text">subbinary2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="findbinary">
+        <output-dir compare="Text">findbinary</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="stringoffset">
+      <compilation-unit name="findbinary2">
+        <output-dir compare="Text">findbinary2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 0d29943..821b6e1 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -57,7 +57,8 @@
                         + "other integer values dictate the number of query execution parallel partitions. The system will "
                         + "fall back to use the number of all available CPU cores in the cluster as the degree of parallelism "
                         + "if the number set by a user is too large or too small"),
-        COMPILER_PREGELIX_HOME(STRING, "~/pregelix", "Pregelix installation root directory");
+        COMPILER_PREGELIX_HOME(STRING, "~/pregelix", "Pregelix installation root directory"),
+        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -91,7 +92,7 @@
 
         @Override
         public boolean hidden() {
-            return this == COMPILER_PREGELIX_HOME;
+            return this == COMPILER_PREGELIX_HOME || this == COMPILER_STRINGOFFSET;
         }
     }
 
@@ -132,4 +133,9 @@
     public String getPregelixHome() {
         return accessor.getString(Option.COMPILER_PREGELIX_HOME);
     }
+
+    public int getStringOffset() {
+        int value = accessor.getInt(Option.COMPILER_STRINGOFFSET);
+        return value > 0 ? 1 : 0;
+    }
 }
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
index 95d0550..1f56c4c 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
@@ -211,8 +211,8 @@
     * `string` : a `string` that might contain the pattern,
     * `string_pattern` : a pattern `string` to be matched.
  * Return Value:
-    * the first position that `string_pattern` appears within `string`,
-      or -1 if it does not appear,
+    * the first position that `string_pattern` appears within `string`
+      (starting at 0), or -1 if it does not appear,
     * `missing` if any argument is a `missing` value,
     * `null` if any argument is a `null` value but no argument is a `missing` value,
     * any other non-string input value will cause a type error.
@@ -306,8 +306,8 @@
     * `string_flag` : (Optional) a `string` with flags to be used during regular expression matching.
         * The following modes are enabled with these flags: dotall (s), multiline (m), case_insensitive (i), and comments and whitespace (x).
  * Return Value:
-    * the first position that the regular expression `string_pattern` appears in `string`,
-      or -1 if it does not appear.
+    * the first position that the regular expression `string_pattern` appears in `string`
+      (starting at 0), or -1 if it does not appear.
     * `missing` if any argument is a `missing` value,
     * `null` if any argument is a `null` value but no argument is a `missing` value,
     * any other non-string input value will cause a type error.
@@ -363,7 +363,7 @@
  * Returns a string formed by repeating the input `string` `n` times.
  * Arguments:
     * `string` : a `string` to be repeated,
-    * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`.
+    * `n` : an `tinyint`/`smallint`/`integer`/`bigint` value - how many times the string should be repeated.
  * Return Value:
     * a string that repeats the input `string` `n` times,
     * `missing` if any argument is a `missing` value,
@@ -471,7 +471,8 @@
  * Returns the substring from the given string `string` based on the given start offset `offset` with the optional `length`.
  * Arguments:
     * `string` : a `string` to be extracted,
-    * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`,
+    * `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`
+                 (starting at 0),
     * `length` : (Optional) an an `tinyint`/`smallint`/`integer`/`bigint` value as the length of the substring.
  * Return Value:
     * a `string` that represents the substring,
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md b/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
index 2902496..82a68be 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
@@ -97,7 +97,7 @@
   * Arguments:
     * `binary` : a `binary` to be extracted,
     * `offset` : a `tinyint`, `smallint`, `integer`, or `bigint` value
-       as the starting offset of the sub binary in `binary`,
+       as the starting offset of the sub binary in `binary` (starting at 0),
     * `length` : (Optional) a `tinyint`, `smallint`, `integer`, or `bigint` value
                   as the length of the sub binary.
   * Return Value:
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
index b8e27e1..97042e2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
@@ -19,11 +19,12 @@
 
 package org.apache.asterix.om.functions;
 
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 
 public interface IFunctionTypeInferer {
-    void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-            throws AlgebricksException;
+    void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+            CompilerProperties compilerProps) throws AlgebricksException;
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java
new file mode 100644
index 0000000..3f8f45f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.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.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+
+public abstract class AbstractStringOffsetConfigurableDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    protected int stringOffset;
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        stringOffset = (int) states[0];
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
index 6f4d116..e9ec211 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
@@ -24,7 +24,8 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -32,7 +33,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class StringPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class StringPositionDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@@ -40,12 +41,19 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StringPositionDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
+        }
     };
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
@@ -55,7 +63,7 @@
                     @Override
                     protected int compute(UTF8StringPointable left, UTF8StringPointable right) throws IOException {
                         int pos = UTF8StringPointable.find(left, right, false);
-                        return pos < 0 ? pos : pos + 1;
+                        return pos < 0 ? pos : pos + baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
index 1bb0fdc..b39c473 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
@@ -24,8 +24,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -33,13 +34,18 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class StringRegExpPositionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class StringRegExpPositionDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StringRegExpPositionDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -48,6 +54,8 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
                 return new AbstractBinaryStringIntEval(ctx, args[0], args[1],
@@ -55,11 +63,10 @@
                     private final RegExpMatcher matcher = new RegExpMatcher();
 
                     @Override
-                    protected int compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr)
-                            throws IOException {
+                    protected int compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr) {
                         matcher.build(srcPtr, patternPtr);
                         int pos = matcher.postion();
-                        return pos < 0 ? pos : pos + 1;
+                        return pos < 0 ? pos : pos + baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
index 35eb3b1..16d428f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
@@ -24,8 +24,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.asterix.runtime.evaluators.functions.utils.RegExpMatcher;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -33,7 +34,7 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 
-public class StringRegExpPositionWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class StringRegExpPositionWithFlagDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
@@ -41,12 +42,19 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new StringRegExpPositionWithFlagDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
+        }
     };
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
@@ -56,10 +64,10 @@
 
                     @Override
                     protected int compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr,
-                            UTF8StringPointable flagPtr) throws IOException {
+                            UTF8StringPointable flagPtr) {
                         matcher.build(srcPtr, patternPtr, flagPtr);
                         int pos = matcher.postion();
-                        return pos < 0 ? pos : pos + 1;
+                        return pos < 0 ? pos : pos + baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
index 5a36942..8ebfb94 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
@@ -26,10 +26,11 @@
 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.ATypeTag;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -43,12 +44,17 @@
 import org.apache.hyracks.data.std.util.UTF8StringBuilder;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class Substring2Descriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class Substring2Descriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new Substring2Descriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -56,6 +62,8 @@
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
@@ -78,7 +86,8 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset) - 1;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset)
+                                - baseOffset;
                         bytes = argString.getByteArray();
                         offset = argString.getStartOffset();
                         int len = argString.getLength();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
index 28bbe08..d2a1203 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
@@ -26,10 +26,11 @@
 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.ATypeTag;
 import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -43,7 +44,7 @@
 import org.apache.hyracks.data.std.util.UTF8StringBuilder;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class SubstringDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class SubstringDescriptor extends AbstractStringOffsetConfigurableDescriptor {
 
     private static final long serialVersionUID = 1L;
 
@@ -52,12 +53,19 @@
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubstringDescriptor();
         }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
+        }
     };
 
     @Override
     public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
+
+            private final int baseOffset = stringOffset;
 
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
@@ -85,7 +93,8 @@
 
                         byte[] bytes = argStart.getByteArray();
                         int offset = argStart.getStartOffset();
-                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset) - 1;
+                        int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset)
+                                - baseOffset;
 
                         bytes = argLen.getByteArray();
                         offset = argLen.getStartOffset();
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
index 827b43e..c88fe25 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
@@ -35,8 +35,9 @@
 public abstract class AbstractFindBinaryEvaluator extends AbstractBinaryScalarEvaluator {
 
     private static final ATypeTag[] EXPECTED_INPUT_TAG = { ATypeTag.BINARY, ATypeTag.BINARY };
-    protected String functionName;
-    protected AMutableInt64 result = new AMutableInt64(-1);
+    protected final int baseOffset;
+    protected final String functionName;
+    protected final AMutableInt64 result = new AMutableInt64(-1);
     protected final ByteArrayPointable textPtr = new ByteArrayPointable();
     protected final ByteArrayPointable wordPtr = new ByteArrayPointable();
 
@@ -45,8 +46,9 @@
             SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
 
     public AbstractFindBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            String functionName) throws HyracksDataException {
+            int baseOffset, String functionName) throws HyracksDataException {
         super(context, copyEvaluatorFactories);
+        this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
 
@@ -64,9 +66,9 @@
         checkTypeMachingThrowsIfNot(functionName, EXPECTED_INPUT_TAG, textTag, wordTag);
         textPtr.set(pointables[0].getByteArray(), pointables[0].getStartOffset() + 1, pointables[0].getLength() - 1);
         wordPtr.set(pointables[1].getByteArray(), pointables[0].getStartOffset() + 1, pointables[1].getLength() - 1);
-        result.setValue(1L + indexOf(textPtr.getByteArray(), textPtr.getContentStartOffset(),
-                textPtr.getContentLength(), wordPtr.getByteArray(), wordPtr.getContentStartOffset(),
-                wordPtr.getContentLength(), fromOffset));
+        int pos = indexOf(textPtr.getByteArray(), textPtr.getContentStartOffset(), textPtr.getContentLength(),
+                wordPtr.getByteArray(), wordPtr.getContentStartOffset(), wordPtr.getContentLength(), fromOffset);
+        result.setValue(pos < 0 ? pos : pos + baseOffset);
         intSerde.serialize(result, dataOutput);
         resultPointable.set(resultStorage);
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
index cd7b7d3..fc049a4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
@@ -36,13 +36,15 @@
 
     private ByteArrayPointable byteArrayPointable = new ByteArrayPointable();
     private byte[] metaBuffer = new byte[5];
+    protected final int baseOffset;
     protected final String functionName;
 
     private static final ATypeTag[] EXPECTED_INPUT_TAGS = { ATypeTag.BINARY, ATypeTag.INTEGER };
 
     public AbstractSubBinaryEvaluator(IHyracksTaskContext context, IScalarEvaluatorFactory[] copyEvaluatorFactories,
-            String functionName) throws HyracksDataException {
+            int baseOffset, String functionName) throws HyracksDataException {
         super(context, copyEvaluatorFactories);
+        this.baseOffset = baseOffset;
         this.functionName = functionName;
     }
 
@@ -67,9 +69,8 @@
 
             int subStart;
 
-            // strange SQL index convention
             subStart = ATypeHierarchy.getIntegerValue(BuiltinFunctions.SUBBINARY_FROM.getName(), 1, startBytes, offset)
-                    - 1;
+                    - baseOffset;
 
             int totalLength = byteArrayPointable.getContentLength();
             int subLength = getSubLength(tuple);
@@ -78,7 +79,10 @@
                 subStart = 0;
             }
 
-            if (subStart >= totalLength || subLength < 0) {
+            if (subStart >= totalLength) {
+                subStart = 0;
+                subLength = 0;
+            } else if (subLength < 0) {
                 subLength = 0;
             } else if (subLength > totalLength // for the IntMax case
                     || subStart + subLength > totalLength) {
@@ -88,6 +92,7 @@
             dataOutput.write(ATypeTag.BINARY.serialize());
             int metaLength = VarLenIntEncoderDecoder.encode(subLength, metaBuffer, 0);
             dataOutput.write(metaBuffer, 0, metaLength);
+
             dataOutput.write(byteArrayPointable.getByteArray(), byteArrayPointable.getContentStartOffset() + subStart,
                     subLength);
         } catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
index 61a24c4..307585d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
@@ -22,7 +22,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -30,12 +32,17 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class FindBinaryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class FindBinaryDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FindBinaryDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -49,11 +56,13 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
-                    protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
+                    protected int getFromOffset(IFrameTupleReference tuple) {
                         return 0;
                     }
                 };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
index 71f8459..243d344 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
@@ -22,8 +22,10 @@
 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.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -31,13 +33,18 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class FindBinaryFromDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class FindBinaryFromDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new FindBinaryFromDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -51,13 +58,15 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractFindBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractFindBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
                     protected int getFromOffset(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
-                                pointables[2].getByteArray(), pointables[2].getStartOffset());
+                                pointables[2].getByteArray(), pointables[2].getStartOffset()) - baseOffset;
                     }
                 };
             }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
index a677072..3b07c6c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
@@ -22,7 +22,9 @@
 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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -30,12 +32,17 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class SubBinaryFromDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class SubBinaryFromDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubBinaryFromDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -49,11 +56,13 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
-                return new AbstractSubBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
-                    protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
+                    protected int getSubLength(IFrameTupleReference tuple) {
                         return Integer.MAX_VALUE;
                     }
                 };
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
index d91c0f4..961edbf 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
@@ -22,8 +22,10 @@
 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.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.AbstractStringOffsetConfigurableDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 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;
@@ -31,12 +33,17 @@
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class SubBinaryFromToDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class SubBinaryFromToDescriptor extends AbstractStringOffsetConfigurableDescriptor {
     private static final long serialVersionUID = 1L;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SubBinaryFromToDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_STRING_OFFSET;
         }
     };
 
@@ -50,10 +57,12 @@
         return new IScalarEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
+            private final int baseOffset = stringOffset;
+
             @Override
             public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
 
-                return new AbstractSubBinaryEvaluator(ctx, args, getIdentifier().getName()) {
+                return new AbstractSubBinaryEvaluator(ctx, args, baseOffset, getIdentifier().getName()) {
                     @Override
                     protected int getSubLength(IFrameTupleReference tuple) throws HyracksDataException {
                         return ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 2,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index 00a5ec8..b8d9778 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -19,6 +19,7 @@
 
 package org.apache.asterix.runtime.functions;
 
+import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -53,16 +54,24 @@
 
     public static final IFunctionTypeInferer SET_EXPRESSION_TYPE = new IFunctionTypeInferer() {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             fd.setImmutableStates(context.getType(expr));
+        }
+    };
+
+    public static final IFunctionTypeInferer SET_STRING_OFFSET = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) {
+            fd.setImmutableStates(compilerProps.getStringOffset());
         }
     };
 
     public static final class CastTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
             IAType rt = TypeCastUtils.getRequiredType(funcExpr);
             IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
@@ -72,8 +81,8 @@
 
     public static final class DeepEqualityTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
             IAType type1 = (IAType) context.getType(f.getArguments().get(1).getValue());
@@ -83,8 +92,8 @@
 
     public static final class FieldAccessByIndexTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             switch (t.getTypeTag()) {
@@ -112,8 +121,8 @@
 
     public static final class FieldAccessNestedTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             AOrderedList fieldPath =
@@ -141,8 +150,8 @@
 
     public static final class GetRecordFieldsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             ATypeTag typeTag = t.getTypeTag();
@@ -158,8 +167,8 @@
 
     public static final class GetRecordFieldValueTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             ATypeTag typeTag = t.getTypeTag();
@@ -175,8 +184,8 @@
 
     public static final class OpenRecordConstructorTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             ARecordType rt = (ARecordType) context.getType(expr);
             fd.setImmutableStates(rt, computeOpenFields((AbstractFunctionCallExpression) expr, rt));
         }
@@ -204,8 +213,8 @@
 
     public static final class RecordAddFieldsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType outType = (IAType) context.getType(expr);
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
@@ -223,8 +232,8 @@
 
     public static final class RecordMergeTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType outType = (IAType) context.getType(expr);
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());
@@ -235,8 +244,8 @@
 
     public static final class RecordPairsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
             IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
             ATypeTag typeTag = t.getTypeTag();
@@ -252,8 +261,8 @@
 
     public static final class RecordRemoveFieldsTypeInferer implements IFunctionTypeInferer {
         @Override
-        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context)
-                throws AlgebricksException {
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
             AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
             IAType outType = (IAType) context.getType(expr);
             IAType type0 = (IAType) context.getType(f.getArguments().get(0).getValue());

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

Gerrit-MessageType: merged
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/8073/ (5/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2: Contrib+1

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/2560/ (4/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/2465/ (2/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/6457/ (8/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/7961/ (9/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Compilation Failed
https://goo.gl/ybeWvW : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3: Code-Review+2

forwarding Till's +2. only updated the commit message.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

I'd like you to reexamine a change.  Please visit

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

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................

[ASTERIXDB-2272][FUN] Count character positions from 0

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

Details:
- Count character positions in strings,binary from 0 instead of 1
- Affects following built-in functions:
  position(), regexp_position(), substr()/subtring(),
  sub_binary(), find-binary()
- Introduce new configuration property “compiler.stringoffset”
  for backward compatibility
  compiler.stringoffset = 0 - count from 0 (new default)
  compiler.stringoffset = 1 - count from 1 (backwards-compatible setting)

Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
A asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/StringOffsetTest.java
A asterixdb/asterix-app/src/test/resources/cc-stringoffset.conf
M asterixdb/asterix-app/src/test/resources/old-optimizerts/queries/j-cust-subplan.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562.aql
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/query-issue562.sqlpp
M asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/query-issue562.ast
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/find/find.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/binary/subbinary/subbinary_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring_01/substring_01.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue562/query-issue562.3.query.aql
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/find/find.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/binary/subbinary/subbinary_01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access/explain_field_access.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/explain/explain_field_access_closed/explain_field_access_closed.2.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr-2/gby-expr-2.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-expr/gby-expr.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/substr/substr.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr04/substr04.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr05/substr05.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr06/substr06.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-1/substring2-1.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-3/substring2-3.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring_01/substring_01.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary/findbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/findbinary2/findbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/position/position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position/regexp_position.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary/subbinary.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/subbinary2/subbinary2.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring/substring.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/stringoffset/substring2/substring2.1.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/exists/exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/not_exists/not_exists.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/query-ASTERIXDB-1329.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q15/q15.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q45/q45.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue562/query-issue562.3.query.sqlpp
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access/explain_field_access.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/explain/explain_field_access_closed/explain_field_access_closed.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/position/position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position/regexp_position.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results/string/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary/findbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/findbinary2/findbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/position/position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position/regexp_position.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/regexp_position_with_flag/regexp_position_with_flag.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary/subbinary.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/subbinary2/subbinary2.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/find/find.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/binary/subbinary/subbinary_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr04/substr04.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr05/substr05.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr06/substr06.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-1/substring2-1.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-3/substring2-3.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring_01/substring_01.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite/tinysocial-suite.24.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/q22_global_sales_opportunity/q22_global_sales_opportunity.3.ast
M asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue562/query-issue562.3.ast
A asterixdb/asterix-app/src/test/resources/runtimets/testsuite_stringoffset.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
M asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
M asterixdb/asterix-doc/src/main/markdown/builtins/3_binary.md
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionTypeInferer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractStringOffsetConfigurableDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringRegExpPositionWithFlagDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractFindBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/AbstractSubBinaryEvaluator.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/FindBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/binary/SubBinaryFromToDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
109 files changed, 872 insertions(+), 247 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3365/ (2/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/2455/ (7/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2: Code-Review+2

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2342/2//COMMIT_MSG
Commit Message:

PS2, Line 14: strings,binary
s/strings,binary/strings, binary/


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 3: Verified+1

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Taewoo Kim <ta...@uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3255/ (10/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2272][FUN] Count character positions from 0

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

Change subject: [ASTERIXDB-2272][FUN] Count character positions from 0
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/514/ (4/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Icdb2c241983d5f592b2695ef16eae9951518d7fd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No