You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2015/04/14 21:09:40 UTC

[3/3] phoenix git commit: PHOENIX-1287 Use the joni byte[] regex engine in place of j.u.regex (Shuxiong Ye)

PHOENIX-1287 Use the joni byte[] regex engine in place of j.u.regex (Shuxiong Ye)


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/3f6b2594
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/3f6b2594
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/3f6b2594

Branch: refs/heads/master
Commit: 3f6b25947d07ea0d7756556dd80e951f12ceda69
Parents: 7ef1718
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Apr 14 12:09:17 2015 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Apr 14 12:09:17 2015 -0700

----------------------------------------------------------------------
 .../src/build/components-major-client.xml       |   2 +
 phoenix-core/pom.xml                            |   5 +
 .../phoenix/end2end/LikeExpressionIT.java       |  88 ++++++++
 .../end2end/RegexpReplaceFunctionIT.java        | 100 +++++++++
 .../phoenix/end2end/RegexpSubstrFunctionIT.java |  43 ++--
 .../phoenix/compile/ExpressionCompiler.java     |  15 +-
 .../expression/ByteBasedLikeExpression.java     |  48 +++++
 .../phoenix/expression/ExpressionType.java      |  16 +-
 .../phoenix/expression/LikeExpression.java      |  64 +++---
 .../expression/StringBasedLikeExpression.java   |  48 +++++
 .../ByteBasedRegexpReplaceFunction.java         |  40 ++++
 .../function/ByteBasedRegexpSplitFunction.java  |  38 ++++
 .../function/ByteBasedRegexpSubstrFunction.java |  38 ++++
 .../function/RegexpReplaceFunction.java         |  38 ++--
 .../function/RegexpSplitFunction.java           |  54 +++--
 .../function/RegexpSubstrFunction.java          |  48 ++---
 .../StringBasedRegexpReplaceFunction.java       |  40 ++++
 .../StringBasedRegexpSplitFunction.java         |  38 ++++
 .../StringBasedRegexpSubstrFunction.java        |  38 ++++
 .../util/regex/AbstractBasePattern.java         |  33 +++
 .../util/regex/AbstractBaseSplitter.java        |  24 +++
 .../expression/util/regex/GuavaSplitter.java    |  54 +++++
 .../expression/util/regex/JONIPattern.java      | 201 +++++++++++++++++++
 .../expression/util/regex/JavaPattern.java      |  93 +++++++++
 .../visitor/CloneExpressionVisitor.java         |   3 +-
 .../phoenix/parse/RegexpReplaceParseNode.java   |  55 +++++
 .../phoenix/parse/RegexpSplitParseNode.java     |  55 +++++
 .../phoenix/parse/RegexpSubstrParseNode.java    |  55 +++++
 .../org/apache/phoenix/query/QueryServices.java |   2 +
 .../phoenix/query/QueryServicesOptions.java     |  14 +-
 .../phoenix/schema/types/PArrayDataType.java    |  91 +++++++++
 .../org/apache/phoenix/util/StringUtil.java     |  68 ++++++-
 .../phoenix/compile/WhereOptimizerTest.java     |  18 +-
 .../phoenix/expression/ILikeExpressionTest.java |  32 ++-
 .../phoenix/expression/LikeExpressionTest.java  |  39 +++-
 .../expression/RegexpReplaceFunctionTest.java   |  81 ++++++++
 .../expression/RegexpSplitFunctionTest.java     |  94 +++++++++
 .../expression/RegexpSubstrFunctionTest.java    |  83 ++++++++
 .../expression/SortOrderExpressionTest.java     |  12 +-
 .../util/regex/PatternPerformanceTest.java      | 144 +++++++++++++
 .../org/apache/phoenix/util/StringUtilTest.java |  32 ++-
 .../java/org/apache/phoenix/util/TestUtil.java  |  28 ++-
 pom.xml                                         |   1 +
 43 files changed, 1952 insertions(+), 161 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-assembly/src/build/components-major-client.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/components-major-client.xml b/phoenix-assembly/src/build/components-major-client.xml
index 768cac0..7a2909b 100644
--- a/phoenix-assembly/src/build/components-major-client.xml
+++ b/phoenix-assembly/src/build/components-major-client.xml
@@ -49,6 +49,8 @@
         <include>org.codehaus.jackson:jackson-core-asl</include>
         <include>commons-collections:commons-collections</include>
         <include>joda-time:joda-time</include>
+        <include>org.jruby.joni:joni</include>
+        <include>org.jruby.jcodings:jcodings</include>
       </includes>
     </dependencySet>
   </dependencySets>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 5e0aff7..45b8d73 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -417,5 +417,10 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-minicluster</artifactId>
     </dependency>
+    <dependency>
+        <groupId>org.jruby.joni</groupId>
+        <artifactId>joni</artifactId>
+        <version>${joni.version}</version>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java
new file mode 100644
index 0000000..1ee0669
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LikeExpressionIT.java
@@ -0,0 +1,88 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.closeStmtAndConn;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class LikeExpressionIT extends BaseHBaseManagedTimeIT {
+    @Before
+    public void doBeforeTestSetup() throws Exception {
+        Connection conn = null;
+        PreparedStatement stmt = null;
+        try {
+            conn = DriverManager.getConnection(getUrl());
+            String ddl;
+            ddl = "CREATE TABLE testTable (k VARCHAR NOT NULL PRIMARY KEY, i INTEGER)";
+            conn.createStatement().execute(ddl);
+            conn.commit();
+        } finally {
+            closeStmtAndConn(stmt, conn);
+        }
+        insertRow(conn, "123n7-app-2-", 1);
+        insertRow(conn, "132n7-App-2-", 2);
+        insertRow(conn, "213n7-app-2-", 4);
+        insertRow(conn, "231n7-App-2-", 8);
+        insertRow(conn, "312n7-app-2-", 16);
+        insertRow(conn, "321n7-App-2-", 32);
+    }
+
+    private void insertRow(Connection conn, String k, int i) throws SQLException {
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO testTable VALUES (?, ?)");
+        stmt.setString(1, k);
+        stmt.setInt(2, i);
+        stmt.executeUpdate();
+        conn.commit();
+    }
+
+    private void testLikeExpression(Connection conn, String likeStr, int numResult, int expectedSum)
+            throws Exception {
+        String cmd = "select k, i from testTable where k like '" + likeStr + "'";
+        Statement stmt = conn.createStatement();
+        ResultSet rs = stmt.executeQuery(cmd);
+        int sum = 0;
+        for (int i = 0; i < numResult; ++i) {
+            assertTrue(rs.next());
+            sum += rs.getInt("i");
+        }
+        assertFalse(rs.next());
+        assertEquals(sum, expectedSum);
+    }
+
+    @Test
+    public void testLikeExpression() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        // wildcard
+        testLikeExpression(conn, "%1%3%7%2%", 3, 7);
+        // CaseSensitive
+        testLikeExpression(conn, "%A%", 3, 42);
+        conn.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java
new file mode 100644
index 0000000..dcc20ff
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpReplaceFunctionIT.java
@@ -0,0 +1,100 @@
+/*
+ * 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.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.GROUPBYTEST_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class RegexpReplaceFunctionIT extends BaseHBaseManagedTimeIT {
+
+    private int id;
+
+    @Before
+    public void doBeforeTestSetup() throws Exception {
+        ensureTableCreated(getUrl(), GROUPBYTEST_NAME);
+        Connection conn = DriverManager.getConnection(getUrl());
+        insertRow(conn, "Report11", 10);
+        insertRow(conn, "Report11", 10);
+        insertRow(conn, "Report22", 30);
+        insertRow(conn, "Report33", 30);
+        conn.commit();
+        conn.close();
+    }
+
+    private void insertRow(Connection conn, String uri, int appcpu) throws SQLException {
+        PreparedStatement statement = conn.prepareStatement("UPSERT INTO " + GROUPBYTEST_NAME + "(id, uri, appcpu) values (?,?,?)");
+        statement.setString(1, "id" + id);
+        statement.setString(2, uri);
+        statement.setInt(3, appcpu);
+        statement.executeUpdate();
+        id++;
+    }
+
+    @Test
+    public void testGroupByScanWithRegexpReplace() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        Statement stmt = conn.createStatement();
+        ResultSet rs = stmt.executeQuery("select REGEXP_REPLACE(uri, '[1-3]+', '*') suburi, sum(appcpu) sumcpu from " + GROUPBYTEST_NAME + " group by suburi");
+        assertTrue(rs.next());
+        assertEquals(rs.getString("suburi"), "Report*");
+        assertEquals(rs.getInt("sumcpu"), 80);
+        assertFalse(rs.next());
+
+        stmt = conn.createStatement();
+        rs = stmt.executeQuery("select REGEXP_REPLACE(uri, '[1-3]+') suburi, sum(appcpu) sumcpu from " + GROUPBYTEST_NAME + " group by suburi");
+        assertTrue(rs.next());
+        assertEquals(rs.getString("suburi"), "Report");
+        assertEquals(rs.getInt("sumcpu"), 80);
+        assertFalse(rs.next());
+
+        conn.close();
+    }
+
+    @Test
+    public void testFilterWithRegexReplace() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        ResultSet rs = conn.createStatement().executeQuery("select id from " + GROUPBYTEST_NAME + " where REGEXP_REPLACE(uri, '[2-3]+', '*') = 'Report*'");
+        assertTrue(rs.next());
+        assertEquals("id2", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("id3", rs.getString(1));
+        assertFalse(rs.next());
+
+        rs = conn.createStatement().executeQuery("select id from " + GROUPBYTEST_NAME + " where REGEXP_REPLACE(uri, '[2-3]+') = 'Report'");
+        assertTrue(rs.next());
+        assertEquals("id2", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("id3", rs.getString(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
index ff4b95e..938fd5d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RegexpSubstrFunctionIT.java
@@ -58,30 +58,37 @@ public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeIT {
         id++;
     }
 
-    @Test
-    public void testGroupByScanWithRegexpSubstr() throws Exception {
-        Connection conn = DriverManager.getConnection(getUrl());
+    private void testGroupByScanWithRegexpSubstr(Connection conn, Integer offset, String exceptedSubstr) throws Exception {
+        String cmd = "select REGEXP_SUBSTR(uri, '[^\\\\?]+'" + ((offset == null) ? "" : ", " + offset.intValue()) +") suburi, sum(appcpu) sumcpu from " + GROUPBYTEST_NAME + " group by suburi";
         Statement stmt = conn.createStatement();
-        ResultSet rs = stmt.executeQuery("select REGEXP_SUBSTR(uri, '[^\\\\?]+') suburi, sum(appcpu) sumcpu from " + GROUPBYTEST_NAME
-            + " group by suburi");
+        ResultSet rs = stmt.executeQuery(cmd);
         assertTrue(rs.next());
-        assertEquals(rs.getString("suburi"), "Report1");
+        assertEquals(rs.getString("suburi"), exceptedSubstr + "1");
         assertEquals(rs.getInt("sumcpu"), 20);
         assertTrue(rs.next());
-        assertEquals(rs.getString("suburi"), "Report2");
+        assertEquals(rs.getString("suburi"), exceptedSubstr + "2");
         assertEquals(rs.getInt("sumcpu"), 30);
         assertTrue(rs.next());
-        assertEquals(rs.getString("suburi"), "Report3");
+        assertEquals(rs.getString("suburi"), exceptedSubstr + "3");
         assertEquals(rs.getInt("sumcpu"), 30);
         assertFalse(rs.next());
-        conn.close();
     }
 
     @Test
-    public void testFilterWithRegexSubstr() throws Exception {
+    public void testGroupByScanWithRegexpSubstr() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        ResultSet rs = conn.createStatement().executeQuery(
-                "select id from " + GROUPBYTEST_NAME + " where REGEXP_SUBSTR(uri, '[^\\\\?]+') = 'Report1'");
+        // Default offset
+        testGroupByScanWithRegexpSubstr(conn, null, "Report");
+        // Positive offset
+        testGroupByScanWithRegexpSubstr(conn, Integer.valueOf(2), "eport");
+        // Negative offset
+        testGroupByScanWithRegexpSubstr(conn, Integer.valueOf(-5), "rt");
+        conn.close();
+    }
+
+    private void testFilterWithRegexSubstr(Connection conn, Integer offset, String exceptedSubstr) throws Exception {
+        String cmd = "select id from " + GROUPBYTEST_NAME + " where REGEXP_SUBSTR(uri, '[^\\\\?]+'"+ ((offset == null) ? "" : ", " + offset.intValue()) +") = '" + exceptedSubstr + "1'";
+        ResultSet rs = conn.createStatement().executeQuery(cmd);
         assertTrue(rs.next());
         assertEquals("id0", rs.getString(1));
         assertTrue(rs.next());
@@ -89,4 +96,16 @@ public class RegexpSubstrFunctionIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
     }
 
+    @Test
+    public void testFilterWithRegexSubstr() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        // Default offset
+        testFilterWithRegexSubstr(conn, null, "Report");
+        // Positive offset
+        testFilterWithRegexSubstr(conn, Integer.valueOf(2), "eport");
+        // Negative offset
+        testFilterWithRegexSubstr(conn, Integer.valueOf(-5), "rt");
+        conn.close();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index 52c67f1..ce95850 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -32,6 +32,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.AndExpression;
 import org.apache.phoenix.expression.ArrayConstructorExpression;
+import org.apache.phoenix.expression.ByteBasedLikeExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
@@ -60,6 +61,7 @@ import org.apache.phoenix.expression.NotExpression;
 import org.apache.phoenix.expression.OrExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.StringBasedLikeExpression;
 import org.apache.phoenix.expression.StringConcatExpression;
 import org.apache.phoenix.expression.TimestampAddExpression;
 import org.apache.phoenix.expression.TimestampSubtractExpression;
@@ -100,6 +102,8 @@ import org.apache.phoenix.parse.StringConcatParseNode;
 import org.apache.phoenix.parse.SubqueryParseNode;
 import org.apache.phoenix.parse.SubtractParseNode;
 import org.apache.phoenix.parse.UnsupportedAllParseNodeVisitor;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.ColumnRef;
@@ -497,7 +501,16 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
                 }
             }
         }
-        Expression expression = LikeExpression.create(children, node.getLikeType());
+        QueryServices services = context.getConnection().getQueryServices();
+        boolean useByteBasedRegex =
+                services.getProps().getBoolean(QueryServices.USE_BYTE_BASED_REGEX_ATTRIB,
+                    QueryServicesOptions.DEFAULT_USE_BYTE_BASED_REGEX);
+        Expression expression;
+        if (useByteBasedRegex) {
+            expression = ByteBasedLikeExpression.create(children, node.getLikeType());
+        } else {
+            expression = StringBasedLikeExpression.create(children, node.getLikeType());
+        }
         if (ExpressionUtil.isConstant(expression)) {
             ImmutableBytesWritable ptr = context.getTempPtr();
             if (!expression.evaluate(null, ptr)) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java
new file mode 100644
index 0000000..4dd4f70
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ByteBasedLikeExpression.java
@@ -0,0 +1,48 @@
+/*
+ * 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.phoenix.expression;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
+import org.apache.phoenix.expression.util.regex.JONIPattern;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
+
+public class ByteBasedLikeExpression extends LikeExpression {
+
+    public ByteBasedLikeExpression() {
+    }
+
+    public ByteBasedLikeExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBasePattern compilePatternSpec(String value) {
+        return new JONIPattern(value);
+    }
+
+    public static LikeExpression create(List<Expression> children, LikeType likeType) {
+        return new ByteBasedLikeExpression(addLikeTypeChild(children, likeType));
+    }
+
+    @Override
+    public LikeExpression clone(List<Expression> children) {
+        return new ByteBasedLikeExpression(children);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index 22778ce..5f598b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -25,6 +25,9 @@ import org.apache.phoenix.expression.function.ArrayAppendFunction;
 import org.apache.phoenix.expression.function.ArrayElemRefExpression;
 import org.apache.phoenix.expression.function.ArrayIndexFunction;
 import org.apache.phoenix.expression.function.ArrayLengthFunction;
+import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
+import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
+import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
 import org.apache.phoenix.expression.function.CeilDateExpression;
 import org.apache.phoenix.expression.function.CeilDecimalExpression;
 import org.apache.phoenix.expression.function.CeilFunction;
@@ -79,6 +82,9 @@ import org.apache.phoenix.expression.function.SignFunction;
 import org.apache.phoenix.expression.function.SqlTypeNameFunction;
 import org.apache.phoenix.expression.function.StddevPopFunction;
 import org.apache.phoenix.expression.function.StddevSampFunction;
+import org.apache.phoenix.expression.function.StringBasedRegexpReplaceFunction;
+import org.apache.phoenix.expression.function.StringBasedRegexpSplitFunction;
+import org.apache.phoenix.expression.function.StringBasedRegexpSubstrFunction;
 import org.apache.phoenix.expression.function.SubstrFunction;
 import org.apache.phoenix.expression.function.SumAggregateFunction;
 import org.apache.phoenix.expression.function.TimezoneOffsetFunction;
@@ -137,6 +143,8 @@ public enum ExpressionType {
     MinAggregateFunction(MinAggregateFunction.class),
     MaxAggregateFunction(MaxAggregateFunction.class),
     LikeExpression(LikeExpression.class),
+    ByteBasedLikeExpression(ByteBasedLikeExpression.class),
+    StringBasedLikeExpression(StringBasedLikeExpression.class),
     NotExpression(NotExpression.class),
     CaseExpression(CaseExpression.class),
     InListExpression(InListExpression.class),
@@ -153,8 +161,12 @@ public enum ExpressionType {
     DecimalDivideExpression(DecimalDivideExpression.class),
     CoalesceFunction(CoalesceFunction.class),
     RegexpReplaceFunction(RegexpReplaceFunction.class),
+    ByteBasedRegexpReplaceFunction(ByteBasedRegexpReplaceFunction.class),
+    StringBasedRegexpReplaceFunction(StringBasedRegexpReplaceFunction.class),
     SQLTypeNameFunction(SqlTypeNameFunction.class),
     RegexpSubstrFunction(RegexpSubstrFunction.class),
+    ByteBasedRegexpSubstrFunction(ByteBasedRegexpSubstrFunction.class),
+    StringBasedRegexpSubstrFunction(StringBasedRegexpSubstrFunction.class),
     StringConcatExpression(StringConcatExpression.class),
     LengthFunction(LengthFunction.class),
     LTrimFunction(LTrimFunction.class),
@@ -199,7 +211,9 @@ public enum ExpressionType {
     SQLIndexTypeFunction(SQLIndexTypeFunction.class),
     ModulusExpression(ModulusExpression.class),
     DistinctValueAggregateFunction(DistinctValueAggregateFunction.class),
-    RegexpSplitFunctiond(RegexpSplitFunction.class),
+    RegexpSplitFunction(RegexpSplitFunction.class),
+    ByteBasedRegexpSplitFunction(ByteBasedRegexpSplitFunction.class),
+    StringBasedRegexpSplitFunction(StringBasedRegexpSplitFunction.class),
     RandomFunction(RandomFunction.class),
     ToTimeFunction(ToTimeFunction.class),
     ToTimestampFunction(ToTimestampFunction.class),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
index 730cffb..52ac969 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LikeExpression.java
@@ -21,11 +21,12 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
-import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.parse.LikeParseNode.LikeType;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
@@ -49,7 +50,7 @@ import com.google.common.collect.Lists;
  * 
  * @since 0.1
  */
-public class LikeExpression extends BaseCompoundExpression {
+public abstract class LikeExpression extends BaseCompoundExpression {
     private static final Logger logger = LoggerFactory.getLogger(LikeExpression.class);
 
     private static final String ZERO_OR_MORE = "\\E.*\\Q";
@@ -195,10 +196,6 @@ public class LikeExpression extends BaseCompoundExpression {
 //        return sb.toString();
 //    }
 
-    public static LikeExpression create(List<Expression> children, LikeType likeType) {
-        return new LikeExpression(addLikeTypeChild(children,likeType));
-    }
-    
     private static final int LIKE_TYPE_INDEX = 2;
     private static final LiteralExpression[] LIKE_TYPE_LITERAL = new LiteralExpression[LikeType.values().length];
     static {
@@ -206,12 +203,12 @@ public class LikeExpression extends BaseCompoundExpression {
             LIKE_TYPE_LITERAL[likeType.ordinal()] = LiteralExpression.newConstant(likeType.name());
         }
     }
-    private Pattern pattern;
+    private AbstractBasePattern pattern;
 
     public LikeExpression() {
     }
 
-    private static List<Expression> addLikeTypeChild(List<Expression> children, LikeType likeType) {
+    protected static List<Expression> addLikeTypeChild(List<Expression> children, LikeType likeType) {
         List<Expression> newChildren = Lists.newArrayListWithExpectedSize(children.size()+1);
         newChildren.addAll(children);
         newChildren.add(LIKE_TYPE_LITERAL[likeType.ordinal()]);
@@ -247,11 +244,14 @@ public class LikeExpression extends BaseCompoundExpression {
         }
     }
 
-    protected Pattern compilePattern (String value) {
-        if (likeType == LikeType.CASE_SENSITIVE)
-            return Pattern.compile(toPattern(value));
-        else
-            return Pattern.compile("(?i)" + toPattern(value));
+    protected abstract AbstractBasePattern compilePatternSpec(String value);
+
+    protected AbstractBasePattern compilePattern(String value) {
+        if (likeType == LikeType.CASE_SENSITIVE) {
+            return compilePatternSpec(toPattern(value));
+        } else {
+            return compilePatternSpec("(?i)" + toPattern(value));
+        }
     }
 
     private Expression getStrExpression() {
@@ -264,36 +264,40 @@ public class LikeExpression extends BaseCompoundExpression {
 
     @Override
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
-        Pattern pattern = this.pattern;
+        AbstractBasePattern pattern = this.pattern;
         if (pattern == null) { // TODO: don't allow? this is going to be slooowwww
             if (!getPatternExpression().evaluate(tuple, ptr)) {
-                if (logger.isDebugEnabled()) {
-                    logger.debug("LIKE is FALSE: pattern is null");
+                if (logger.isTraceEnabled()) {
+                    logger.trace("LIKE is FALSE: pattern is null");
                 }
                 return false;
             }
             String value = (String) PVarchar.INSTANCE.toObject(ptr, getPatternExpression().getSortOrder());
             pattern = compilePattern(value);
-            if (logger.isDebugEnabled()) {
-                logger.debug("LIKE pattern is expression: " + pattern.pattern());
+            if (logger.isTraceEnabled()) {
+                logger.trace("LIKE pattern is expression: " + pattern.pattern());
             }
         }
 
-        if (!getStrExpression().evaluate(tuple, ptr)) {
-            if (logger.isDebugEnabled()) {
-                logger.debug("LIKE is FALSE: child expression is null");
+        Expression strExpression = getStrExpression();
+        SortOrder strSortOrder = strExpression.getSortOrder();
+        PVarchar strDataType = PVarchar.INSTANCE;
+        if (!strExpression.evaluate(tuple, ptr)) {
+            if (logger.isTraceEnabled()) {
+                logger.trace("LIKE is FALSE: child expression is null");
             }
             return false;
         }
-        if (ptr.getLength() == 0) {
-            return true;
-        }
 
-        String value = (String) PVarchar.INSTANCE.toObject(ptr, getStrExpression().getSortOrder());
-        boolean matched = pattern.matcher(value).matches();
-        ptr.set(matched ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
-        if (logger.isDebugEnabled()) {
-            logger.debug("LIKE(value='" + value + "'pattern='" + pattern.pattern() + "' is " + matched);
+        String value = null;
+        if (logger.isTraceEnabled()) {
+            value = (String) strDataType.toObject(ptr, strSortOrder);
+        }
+        strDataType.coerceBytes(ptr, strDataType, strSortOrder, SortOrder.ASC);
+        pattern.matches(ptr, ptr);
+        if (logger.isTraceEnabled()) {
+            boolean matched = ((Boolean) PBoolean.INSTANCE.toObject(ptr)).booleanValue();
+            logger.trace("LIKE(value='" + value + "'pattern='" + pattern.pattern() + "' is " + matched);
         }
         return true;
     }
@@ -348,4 +352,6 @@ public class LikeExpression extends BaseCompoundExpression {
     public String toString() {
         return (children.get(0) + " LIKE " + children.get(1));
     }
+
+    abstract public LikeExpression clone(List<Expression> children);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java
new file mode 100644
index 0000000..e2afea2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/StringBasedLikeExpression.java
@@ -0,0 +1,48 @@
+/*
+ * 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.phoenix.expression;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
+import org.apache.phoenix.expression.util.regex.JavaPattern;
+import org.apache.phoenix.parse.LikeParseNode.LikeType;
+
+public class StringBasedLikeExpression extends LikeExpression {
+
+    public StringBasedLikeExpression() {
+    }
+
+    public StringBasedLikeExpression(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBasePattern compilePatternSpec(String value) {
+        return new JavaPattern(value);
+    }
+
+    public static LikeExpression create(List<Expression> children, LikeType likeType) {
+        return new StringBasedLikeExpression(addLikeTypeChild(children, likeType));
+    }
+
+    @Override
+    public LikeExpression clone(List<Expression> children) {
+        return new StringBasedLikeExpression(children);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java
new file mode 100644
index 0000000..0d6543c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpReplaceFunction.java
@@ -0,0 +1,40 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
+import org.apache.phoenix.expression.util.regex.JONIPattern;
+
+public class ByteBasedRegexpReplaceFunction extends RegexpReplaceFunction {
+
+    public ByteBasedRegexpReplaceFunction() {
+    }
+
+    public ByteBasedRegexpReplaceFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBasePattern compilePatternSpec(String value) {
+        return new JONIPattern(value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java
new file mode 100644
index 0000000..062713e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSplitFunction.java
@@ -0,0 +1,38 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.util.regex.AbstractBaseSplitter;
+import org.apache.phoenix.expression.util.regex.JONIPattern;
+
+public class ByteBasedRegexpSplitFunction extends RegexpSplitFunction {
+    public ByteBasedRegexpSplitFunction() {
+    }
+
+    public ByteBasedRegexpSplitFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBaseSplitter compilePatternSpec(String value) {
+        return new JONIPattern(value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java
new file mode 100644
index 0000000..7ee99bf
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ByteBasedRegexpSubstrFunction.java
@@ -0,0 +1,38 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
+import org.apache.phoenix.expression.util.regex.JONIPattern;
+
+public class ByteBasedRegexpSubstrFunction extends RegexpSubstrFunction {
+    public ByteBasedRegexpSubstrFunction() {
+    }
+
+    public ByteBasedRegexpSubstrFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBasePattern compilePatternSpec(String value) {
+        return new JONIPattern(value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
index 3f470a9..f22c978 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpReplaceFunction.java
@@ -20,17 +20,18 @@ package org.apache.phoenix.expression.function;
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.List;
-import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
+import org.apache.phoenix.parse.RegexpReplaceParseNode;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.tuple.Tuple;
 
 
 /**
@@ -48,15 +49,16 @@ import org.apache.phoenix.schema.tuple.Tuple;
  * 
  * @since 0.1
  */
-@BuiltInFunction(name=RegexpReplaceFunction.NAME, args= {
+@BuiltInFunction(name=RegexpReplaceFunction.NAME,
+    nodeClass = RegexpReplaceParseNode.class, args= {
     @Argument(allowedTypes={PVarchar.class}),
     @Argument(allowedTypes={PVarchar.class}),
     @Argument(allowedTypes={PVarchar.class},defaultValue="null")} )
-public class RegexpReplaceFunction extends ScalarFunction {
+public abstract class RegexpReplaceFunction extends ScalarFunction {
     public static final String NAME = "REGEXP_REPLACE";
 
     private boolean hasReplaceStr;
-    private Pattern pattern;
+    private AbstractBasePattern pattern;
     
     public RegexpReplaceFunction() { }
 
@@ -66,11 +68,13 @@ public class RegexpReplaceFunction extends ScalarFunction {
         init();
     }
 
+    protected abstract AbstractBasePattern compilePatternSpec(String value);
+
     private void init() {
         hasReplaceStr = ((LiteralExpression)getReplaceStrExpression()).getValue() != null;
         Object patternString = ((LiteralExpression)children.get(1)).getValue();
         if (patternString != null) {
-            pattern = Pattern.compile((String)patternString);
+            pattern = compilePatternSpec((String) patternString);
         }
     }
 
@@ -84,22 +88,20 @@ public class RegexpReplaceFunction extends ScalarFunction {
         if (!sourceStrExpression.evaluate(tuple, ptr)) {
             return false;
         }
-        String sourceStr = (String) PVarchar.INSTANCE.toObject(ptr, sourceStrExpression.getSortOrder());
-        if (sourceStr == null) {
-            return false;
-        }
-        String replaceStr;
+        if (ptr == null) return false;
+        PVarchar type = PVarchar.INSTANCE;
+        type.coerceBytes(ptr, type, sourceStrExpression.getSortOrder(), SortOrder.ASC);
+        ImmutableBytesWritable replacePtr = new ImmutableBytesWritable();
         if (hasReplaceStr) {
-            Expression replaceStrExpression = this.getReplaceStrExpression();
-            if (!replaceStrExpression.evaluate(tuple, ptr)) {
+            Expression replaceStrExpression = getReplaceStrExpression();
+            if (!replaceStrExpression.evaluate(tuple, replacePtr)) {
                 return false;
             }
-            replaceStr = (String) PVarchar.INSTANCE.toObject(ptr, replaceStrExpression.getSortOrder());
+            type.coerceBytes(replacePtr, type, replaceStrExpression.getSortOrder(), SortOrder.ASC);
         } else {
-            replaceStr = "";
+            replacePtr.set(type.toBytes(""));
         }
-        String replacedStr = pattern.matcher(sourceStr).replaceAll(replaceStr);
-        ptr.set(PVarchar.INSTANCE.toBytes(replacedStr));
+        pattern.replaceAll(ptr, replacePtr, ptr);
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSplitFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSplitFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSplitFunction.java
index 89c7c9e..b43dec9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSplitFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSplitFunction.java
@@ -24,17 +24,16 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.util.regex.AbstractBaseSplitter;
 import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.parse.RegexpSplitParseNode;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.schema.types.PVarcharArray;
-import org.apache.phoenix.schema.types.PhoenixArray;
-import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
 
-import com.google.common.base.Splitter;
-import com.google.common.collect.Lists;
-
 /**
  * Function to split a string value into a {@code VARCHAR_ARRAY}.
  * <p>
@@ -46,14 +45,15 @@ import com.google.common.collect.Lists;
  *
  * The function returns a {@link org.apache.phoenix.schema.types.PVarcharArray}
  */
- @FunctionParseNode.BuiltInFunction(name=RegexpSplitFunction.NAME, args= {
+ @FunctionParseNode.BuiltInFunction(name=RegexpSplitFunction.NAME,
+        nodeClass = RegexpSplitParseNode.class, args= {
         @FunctionParseNode.Argument(allowedTypes={PVarchar.class}),
         @FunctionParseNode.Argument(allowedTypes={PVarchar.class})})
-public class RegexpSplitFunction extends ScalarFunction {
+public abstract class RegexpSplitFunction extends ScalarFunction {
 
     public static final String NAME = "REGEXP_SPLIT";
 
-    private Splitter initializedSplitter = null;
+    private AbstractBaseSplitter initializedSplitter = null;
 
     public RegexpSplitFunction() {}
 
@@ -67,11 +67,13 @@ public class RegexpSplitFunction extends ScalarFunction {
         if (patternExpression instanceof LiteralExpression) {
             Object patternValue = ((LiteralExpression) patternExpression).getValue();
             if (patternValue != null) {
-                initializedSplitter = Splitter.onPattern(patternValue.toString());
+                initializedSplitter = compilePatternSpec(patternValue.toString());
             }
         }
     }
 
+    protected abstract AbstractBaseSplitter compilePatternSpec(String value);
+
     @Override
     public void readFields(DataInput input) throws IOException {
         super.readFields(input);
@@ -90,38 +92,28 @@ public class RegexpSplitFunction extends ScalarFunction {
         }
 
         Expression sourceStrExpression = children.get(0);
-        String sourceStr = (String) PVarchar.INSTANCE.toObject(ptr, sourceStrExpression.getSortOrder());
-        if (sourceStr == null) { // sourceStr evaluated to null
-            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
-            return true;
-        }
+        PVarchar type = PVarchar.INSTANCE;
+        type.coerceBytes(ptr, type, sourceStrExpression.getSortOrder(), SortOrder.ASC);
 
-        return split(tuple, ptr, sourceStr);
-    }
-
-    private boolean split(Tuple tuple, ImmutableBytesWritable ptr, String sourceStr) {
-        Splitter splitter = initializedSplitter;
+        AbstractBaseSplitter splitter = initializedSplitter;
         if (splitter == null) {
+            ImmutableBytesWritable tmpPtr = new ImmutableBytesWritable();
             Expression patternExpression = children.get(1);
-            if (!patternExpression.evaluate(tuple, ptr)) {
+            if (!patternExpression.evaluate(tuple, tmpPtr)) {
                 return false;
             }
-            if (ptr.getLength() == 0) {
-                return true; // ptr is already set to null
+            if (tmpPtr.getLength() == 0) {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                return true; // set ptr to null
             }
-
-            String patternStr = (String) PVarchar.INSTANCE.toObject(
-                    ptr, patternExpression.getSortOrder());
-            splitter = Splitter.onPattern(patternStr);
+            String patternStr =
+                    (String) PVarchar.INSTANCE.toObject(tmpPtr, patternExpression.getSortOrder());
+            splitter = compilePatternSpec(patternStr);
         }
 
-        List<String> splitStrings = Lists.newArrayList(splitter.split(sourceStr));
-        PhoenixArray splitArray = new PhoenixArray(PVarchar.INSTANCE, splitStrings.toArray());
-        ptr.set(PVarcharArray.INSTANCE.toBytes(splitArray));
-        return true;
+        return splitter.split(ptr, ptr);
     }
 
-
     @Override
     public PDataType getDataType() {
         return PVarcharArray.INSTANCE;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
index 93d8706..430b444 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RegexpSubstrFunction.java
@@ -20,19 +20,19 @@ package org.apache.phoenix.expression.function;
 import java.io.DataInput;
 import java.io.IOException;
 import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
-import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.parse.RegexpSubstrParseNode;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.tuple.Tuple;
-import org.apache.phoenix.util.ByteUtil;
 
 
 /**
@@ -47,17 +47,20 @@ import org.apache.phoenix.util.ByteUtil;
  * 
  * @since 0.1
  */
-@BuiltInFunction(name=RegexpSubstrFunction.NAME, args={
+@BuiltInFunction(name=RegexpSubstrFunction.NAME,
+    nodeClass = RegexpSubstrParseNode.class, args={
     @Argument(allowedTypes={PVarchar.class}),
     @Argument(allowedTypes={PVarchar.class}),
     @Argument(allowedTypes={PLong.class}, defaultValue="1")} )
-public class RegexpSubstrFunction extends PrefixFunction {
+public abstract class RegexpSubstrFunction extends PrefixFunction {
     public static final String NAME = "REGEXP_SUBSTR";
 
-    private Pattern pattern;
+    private AbstractBasePattern pattern;
     private boolean isOffsetConstant;
     private Integer maxLength;
 
+    private static final PDataType TYPE = PVarchar.INSTANCE;
+
     public RegexpSubstrFunction() { }
 
     public RegexpSubstrFunction(List<Expression> children) {
@@ -65,10 +68,12 @@ public class RegexpSubstrFunction extends PrefixFunction {
         init();
     }
 
+    protected abstract AbstractBasePattern compilePatternSpec(String value);
+
     private void init() {
         Object patternString = ((LiteralExpression)children.get(1)).getValue();
         if (patternString != null) {
-            pattern = Pattern.compile((String)patternString);
+            pattern = compilePatternSpec((String) patternString);
         }
         // If the source string has a fixed width, then the max length would be the length 
         // of the source string minus the offset, or the absolute value of the offset if 
@@ -95,13 +100,11 @@ public class RegexpSubstrFunction extends PrefixFunction {
         if (pattern == null) {
             return false;
         }
-        if (!getSourceStrExpression().evaluate(tuple, ptr)) {
-            return false;
-        }
-        String sourceStr = (String) PVarchar.INSTANCE.toObject(ptr, getSourceStrExpression().getSortOrder());
-        if (sourceStr == null) {
+        ImmutableBytesWritable srcPtr = new ImmutableBytesWritable();
+        if (!getSourceStrExpression().evaluate(tuple, srcPtr)) {
             return false;
         }
+        TYPE.coerceBytes(srcPtr, TYPE, getSourceStrExpression().getSortOrder(), SortOrder.ASC);
 
         Expression offsetExpression = getOffsetExpression();
         if (!offsetExpression.evaluate(tuple, ptr)) {
@@ -109,25 +112,10 @@ public class RegexpSubstrFunction extends PrefixFunction {
         }
         int offset = offsetExpression.getDataType().getCodec().decodeInt(ptr, offsetExpression.getSortOrder());
 
-        int strlen = sourceStr.length();
         // Account for 1 versus 0-based offset
         offset = offset - (offset <= 0 ? 0 : 1);
-        if (offset < 0) { // Offset < 0 means get from end
-            offset = strlen + offset;
-        }
-        if (offset < 0 || offset >= strlen) {
-            return false;
-        }
 
-        Matcher matcher = pattern.matcher(sourceStr);
-        boolean hasSubString = matcher.find(offset);
-        if (!hasSubString) {
-            ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
-            return true;
-        }
-        String subString = matcher.group();
-        ptr.set(PVarchar.INSTANCE.toBytes(subString));
-        return true;
+        return pattern.substr(srcPtr, offset, ptr);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java
new file mode 100644
index 0000000..9aaec70
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpReplaceFunction.java
@@ -0,0 +1,40 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
+import org.apache.phoenix.expression.util.regex.JavaPattern;
+
+public class StringBasedRegexpReplaceFunction extends RegexpReplaceFunction {
+
+    public StringBasedRegexpReplaceFunction() {
+    }
+
+    public StringBasedRegexpReplaceFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBasePattern compilePatternSpec(String value) {
+        return new JavaPattern(value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSplitFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSplitFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSplitFunction.java
new file mode 100644
index 0000000..77321c2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSplitFunction.java
@@ -0,0 +1,38 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.util.regex.AbstractBaseSplitter;
+import org.apache.phoenix.expression.util.regex.GuavaSplitter;
+
+public class StringBasedRegexpSplitFunction extends RegexpSplitFunction {
+    public StringBasedRegexpSplitFunction() {
+    }
+
+    public StringBasedRegexpSplitFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBaseSplitter compilePatternSpec(String value) {
+        return new GuavaSplitter(value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java
new file mode 100644
index 0000000..253db36
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/StringBasedRegexpSubstrFunction.java
@@ -0,0 +1,38 @@
+/*
+ * 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.phoenix.expression.function;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.util.regex.AbstractBasePattern;
+import org.apache.phoenix.expression.util.regex.JavaPattern;
+
+public class StringBasedRegexpSubstrFunction extends RegexpSubstrFunction {
+    public StringBasedRegexpSubstrFunction() {
+    }
+
+    public StringBasedRegexpSubstrFunction(List<Expression> children) {
+        super(children);
+    }
+
+    @Override
+    protected AbstractBasePattern compilePatternSpec(String value) {
+        return new JavaPattern(value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java
new file mode 100644
index 0000000..27b47a0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBasePattern.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression.util.regex;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+public abstract class AbstractBasePattern {
+
+    public abstract void matches(ImmutableBytesWritable srcPtr, ImmutableBytesWritable outPtr);
+
+    public abstract void replaceAll(ImmutableBytesWritable srcPtr,
+            ImmutableBytesWritable replacePtr, ImmutableBytesWritable outPtr);
+
+    public abstract boolean substr(ImmutableBytesWritable srcPtr, int offsetInStr,
+            ImmutableBytesWritable outPtr);
+
+    public abstract String pattern();
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBaseSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBaseSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBaseSplitter.java
new file mode 100644
index 0000000..323eed0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/AbstractBaseSplitter.java
@@ -0,0 +1,24 @@
+/*
+ * 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.phoenix.expression.util.regex;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+public abstract interface AbstractBaseSplitter {
+    public abstract boolean split(ImmutableBytesWritable srcPtr, ImmutableBytesWritable outPtr);
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/GuavaSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/GuavaSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/GuavaSplitter.java
new file mode 100644
index 0000000..325919e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/GuavaSplitter.java
@@ -0,0 +1,54 @@
+/*
+ * 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.phoenix.expression.util.regex;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PVarcharArray;
+import org.apache.phoenix.schema.types.PhoenixArray;
+import org.apache.phoenix.util.ByteUtil;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+
+public class GuavaSplitter implements AbstractBaseSplitter {
+    private final Splitter splitter;
+
+    public GuavaSplitter(String patternString) {
+        if (patternString != null) {
+            splitter = Splitter.onPattern(patternString);
+        } else {
+            splitter = null;
+        }
+    }
+
+    @Override
+    public boolean split(ImmutableBytesWritable srcPtr, ImmutableBytesWritable outPtr) {
+        String sourceStr = (String) PVarchar.INSTANCE.toObject(srcPtr);
+        if (sourceStr == null) { // sourceStr evaluated to null
+            outPtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+        } else {
+            List<String> splitStrings = Lists.newArrayList(splitter.split(sourceStr));
+            PhoenixArray splitArray = new PhoenixArray(PVarchar.INSTANCE, splitStrings.toArray());
+            outPtr.set(PVarcharArray.INSTANCE.toBytes(splitArray));
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
new file mode 100644
index 0000000..5c0b1bc
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
@@ -0,0 +1,201 @@
+/*
+ * 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.phoenix.expression.util.regex;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PArrayDataType.PArrayDataTypeBytesArrayBuilder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.jcodings.Encoding;
+import org.jcodings.specific.UTF8Encoding;
+import org.joni.Matcher;
+import org.joni.Option;
+import org.joni.Regex;
+import org.joni.Syntax;
+
+import com.google.common.base.Preconditions;
+
+public class JONIPattern extends AbstractBasePattern implements AbstractBaseSplitter {
+
+    private final Regex pattern;
+    private final String patternString;
+
+    public JONIPattern(String patternString) {
+        this(patternString, 0);
+    }
+
+    public JONIPattern(String patternString, int flags) {
+        this(patternString, flags, UTF8Encoding.INSTANCE);
+    }
+
+    public JONIPattern(String patternString, int flags, Encoding coding) {
+        this.patternString = patternString;
+        if (patternString != null) {
+            byte[] bytes = patternString.getBytes();
+            pattern = new Regex(bytes, 0, bytes.length, flags, coding, Syntax.Java);
+        } else {
+            pattern = null;
+        }
+    }
+
+    @Override
+    public void matches(ImmutableBytesWritable srcPtr, ImmutableBytesWritable outPtr) {
+        Preconditions.checkNotNull(srcPtr);
+        Preconditions.checkNotNull(outPtr);
+        boolean ret = matches(srcPtr.get(), srcPtr.getOffset(), srcPtr.getLength());
+        outPtr.set(ret ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
+    }
+
+    private boolean matches(byte[] bytes, int offset, int len) {
+        int range = offset + len;
+        Matcher matcher = pattern.matcher(bytes, offset, range);
+        int ret = matcher.match(offset, range, Option.DEFAULT);
+        return len == ret;
+    }
+
+    @Override
+    public String pattern() {
+        return patternString;
+    }
+
+    @Override
+    public void replaceAll(ImmutableBytesWritable srcPtr, ImmutableBytesWritable replacePtr,
+            ImmutableBytesWritable replacedPtr) {
+        Preconditions.checkNotNull(srcPtr);
+        Preconditions.checkNotNull(replacePtr);
+        Preconditions.checkNotNull(replacedPtr);
+        byte[] replacedBytes =
+                replaceAll(srcPtr.get(), srcPtr.getOffset(), srcPtr.getLength(), replacePtr.get(),
+                    replacePtr.getOffset(), replacePtr.getLength());
+        replacedPtr.set(replacedBytes);
+    }
+
+    private byte[] replaceAll(byte[] srcBytes, int srcOffset, int srcLen, byte[] replaceBytes,
+            int replaceOffset, int replaceLen) {
+        class PairInt {
+            public int begin, end;
+
+            public PairInt(int begin, int end) {
+                this.begin = begin;
+                this.end = end;
+            }
+        }
+        int srcRange = srcOffset + srcLen;
+        Matcher matcher = pattern.matcher(srcBytes, 0, srcRange);
+        int cur = srcOffset;
+        List<PairInt> searchResults = new LinkedList<PairInt>();
+        int totalBytesNeeded = 0;
+        while (true) {
+            int nextCur = matcher.search(cur, srcRange, Option.DEFAULT);
+            if (nextCur < 0) {
+                totalBytesNeeded += srcRange - cur;
+                break;
+            }
+            searchResults.add(new PairInt(matcher.getBegin(), matcher.getEnd()));
+            totalBytesNeeded += (nextCur - cur) + replaceLen;
+            cur = matcher.getEnd();
+        }
+        byte[] ret = new byte[totalBytesNeeded];
+        int curPosInSrc = srcOffset, curPosInRet = 0;
+        for (PairInt pair : searchResults) {
+            System.arraycopy(srcBytes, curPosInSrc, ret, curPosInRet, pair.begin - curPosInSrc);
+            curPosInRet += pair.begin - curPosInSrc;
+            System.arraycopy(replaceBytes, replaceOffset, ret, curPosInRet, replaceLen);
+            curPosInRet += replaceLen;
+            curPosInSrc = pair.end;
+        }
+        System.arraycopy(srcBytes, curPosInSrc, ret, curPosInRet, srcRange - curPosInSrc);
+        return ret;
+    }
+
+    @Override
+    public boolean substr(ImmutableBytesWritable srcPtr, int offsetInStr,
+            ImmutableBytesWritable outPtr) {
+        Preconditions.checkNotNull(srcPtr);
+        Preconditions.checkNotNull(outPtr);
+        int offsetInBytes = StringUtil.calculateUTF8Offset(srcPtr.get(), srcPtr.getOffset(),
+            srcPtr.getLength(), SortOrder.ASC, offsetInStr);
+        if (offsetInBytes < 0) return false;
+        substr(srcPtr.get(), offsetInBytes, srcPtr.getOffset() + srcPtr.getLength(), outPtr);
+        return true;
+    }
+
+    private boolean substr(byte[] srcBytes, int offset, int range, ImmutableBytesWritable outPtr) {
+        Matcher matcher = pattern.matcher(srcBytes, 0, range);
+        boolean ret = matcher.search(offset, range, Option.DEFAULT) >= 0;
+        if (ret) {
+            int len = matcher.getEnd() - matcher.getBegin();
+            outPtr.set(srcBytes, matcher.getBegin(), len);
+        } else {
+            outPtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+        }
+        return ret;
+    }
+
+    @Override
+    public boolean split(ImmutableBytesWritable srcPtr, ImmutableBytesWritable outPtr) {
+        return split(srcPtr.get(), srcPtr.getOffset(), srcPtr.getLength(), outPtr);
+    }
+
+    private boolean
+            split(byte[] srcBytes, int srcOffset, int srcLen, ImmutableBytesWritable outPtr) {
+        PArrayDataTypeBytesArrayBuilder builder =
+                new PArrayDataTypeBytesArrayBuilder(PVarchar.INSTANCE, SortOrder.ASC);
+        int srcRange = srcOffset + srcLen;
+        Matcher matcher = pattern.matcher(srcBytes, 0, srcRange);
+        int cur = srcOffset;
+        boolean append;
+        while (true) {
+            int nextCur = matcher.search(cur, srcRange, Option.DEFAULT);
+            if (nextCur < 0) {
+                append = builder.appendElem(srcBytes, cur, srcRange - cur);
+                if (!append) return false;
+                break;
+            }
+
+            // To handle the following case, which adds null at first.
+            // REGEXP_SPLIT("12ONE34TWO56THREE78","[0-9]+")={null, "ONE", "TWO", "THREE", null}
+            if (cur == matcher.getBegin()) {
+                builder.appendElem(srcBytes, cur, 0);
+            }
+
+            if (cur < matcher.getBegin()) {
+                append = builder.appendElem(srcBytes, cur, matcher.getBegin() - cur);
+                if (!append) return false;
+            }
+            cur = matcher.getEnd();
+
+            // To handle the following case, which adds null at last.
+            // REGEXP_SPLIT("12ONE34TWO56THREE78","[0-9]+")={null, "ONE", "TWO", "THREE", null}
+            if (cur == srcRange) {
+                builder.appendElem(srcBytes, cur, 0);
+                break;
+            }
+        }
+        byte[] bytes = builder.getBytesAndClose();
+        if (bytes == null) return false;
+        outPtr.set(bytes);
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java
new file mode 100644
index 0000000..be1188c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JavaPattern.java
@@ -0,0 +1,93 @@
+/*
+ * 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.phoenix.expression.util.regex;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.ByteUtil;
+
+import com.google.common.base.Preconditions;
+
+public class JavaPattern extends AbstractBasePattern {
+
+    private final Pattern pattern;
+
+    public JavaPattern(String patternString) {
+        this(patternString, 0);
+    }
+
+    public JavaPattern(String patternString, int flags) {
+        if (patternString != null) {
+            pattern = Pattern.compile(patternString, flags);
+        } else {
+            pattern = null;
+        }
+    }
+
+    @Override
+    public void matches(ImmutableBytesWritable srcPtr, ImmutableBytesWritable outPtr) {
+        Preconditions.checkNotNull(srcPtr);
+        Preconditions.checkNotNull(outPtr);
+        String matcherSourceStr = (String) PVarchar.INSTANCE.toObject(srcPtr);
+        if (srcPtr.get().length == 0 && matcherSourceStr == null) matcherSourceStr = "";
+        boolean ret = pattern.matcher(matcherSourceStr).matches();
+        outPtr.set(ret ? PDataType.TRUE_BYTES : PDataType.FALSE_BYTES);
+    }
+
+    @Override
+    public String pattern() {
+        return pattern.pattern();
+    }
+
+    @Override
+    public void replaceAll(ImmutableBytesWritable srcPtr, ImmutableBytesWritable replacePtr,
+            ImmutableBytesWritable replacedPtr) {
+        Preconditions.checkNotNull(srcPtr);
+        Preconditions.checkNotNull(replacePtr);
+        Preconditions.checkNotNull(replacedPtr);
+        String sourceStr = (String) PVarchar.INSTANCE.toObject(srcPtr);
+        String replaceStr = (String) PVarchar.INSTANCE.toObject(replacePtr);
+        if (srcPtr.get().length == 0 && sourceStr == null) sourceStr = "";
+        if (replacePtr.get().length == 0 && replaceStr == null) replaceStr = "";
+        String replacedStr = pattern.matcher(sourceStr).replaceAll(replaceStr);
+        replacedPtr.set(PVarchar.INSTANCE.toBytes(replacedStr));
+    }
+
+    @Override
+    public boolean substr(ImmutableBytesWritable srcPtr, int offsetInStr,
+            ImmutableBytesWritable outPtr) {
+        Preconditions.checkNotNull(srcPtr);
+        Preconditions.checkNotNull(outPtr);
+        String sourceStr = (String) PVarchar.INSTANCE.toObject(srcPtr);
+        if (srcPtr.get().length == 0 && sourceStr == null) sourceStr = "";
+        if (offsetInStr < 0) offsetInStr += sourceStr.length();
+        if (offsetInStr < 0 || offsetInStr >= sourceStr.length()) return false;
+        Matcher matcher = pattern.matcher(sourceStr);
+        boolean ret = matcher.find(offsetInStr);
+        if (ret) {
+            outPtr.set(PVarchar.INSTANCE.toBytes(matcher.group()));
+        } else {
+            outPtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
index f415b01..e6ede7c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
@@ -107,7 +107,8 @@ public class CloneExpressionVisitor extends TraverseAllExpressionVisitor<Express
 
     @Override
     public Expression visitLeave(LikeExpression node, List<Expression> l) {
-        return Determinism.PER_INVOCATION.compareTo(node.getDeterminism()) > 0 ? node :  new LikeExpression(l);
+        return Determinism.PER_INVOCATION.compareTo(node.getDeterminism()) > 0 ? node : node
+                .clone(l);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpReplaceParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpReplaceParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpReplaceParseNode.java
new file mode 100644
index 0000000..4d98405
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpReplaceParseNode.java
@@ -0,0 +1,55 @@
+/*
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
+import org.apache.phoenix.expression.function.RegexpReplaceFunction;
+import org.apache.phoenix.expression.function.StringBasedRegexpReplaceFunction;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+
+/**
+ * Parse node corresponding to {@link RegexpReplaceFunction}. It also acts as a factory for creating
+ * the right kind of RegexpReplaceFunction according to setting in
+ * QueryServices.USE_BYTE_BASED_REGEX_ATTRIB
+ */
+public class RegexpReplaceParseNode extends FunctionParseNode {
+
+    RegexpReplaceParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public Expression create(List<Expression> children, StatementContext context)
+            throws SQLException {
+        QueryServices services = context.getConnection().getQueryServices();
+        boolean useByteBasedRegex =
+                services.getProps().getBoolean(QueryServices.USE_BYTE_BASED_REGEX_ATTRIB,
+                    QueryServicesOptions.DEFAULT_USE_BYTE_BASED_REGEX);
+        if (useByteBasedRegex) {
+            return new ByteBasedRegexpReplaceFunction(children);
+        } else {
+            return new StringBasedRegexpReplaceFunction(children);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3f6b2594/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpSplitParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpSplitParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpSplitParseNode.java
new file mode 100644
index 0000000..74bee07
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/RegexpSplitParseNode.java
@@ -0,0 +1,55 @@
+/*
+ * 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.phoenix.parse;
+
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
+import org.apache.phoenix.expression.function.RegexpSplitFunction;
+import org.apache.phoenix.expression.function.StringBasedRegexpSplitFunction;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+
+/**
+ * Parse node corresponding to {@link RegexpSplitFunction}. It also acts as a factory for creating
+ * the right kind of RegexpSplitFunction according to setting in
+ * QueryServices.USE_BYTE_BASED_REGEX_ATTRIB
+ */
+public class RegexpSplitParseNode extends FunctionParseNode {
+
+    RegexpSplitParseNode(String name, List<ParseNode> children, BuiltInFunctionInfo info) {
+        super(name, children, info);
+    }
+
+    @Override
+    public Expression create(List<Expression> children, StatementContext context)
+            throws SQLException {
+        QueryServices services = context.getConnection().getQueryServices();
+        boolean useByteBasedRegex =
+                services.getProps().getBoolean(QueryServices.USE_BYTE_BASED_REGEX_ATTRIB,
+                    QueryServicesOptions.DEFAULT_USE_BYTE_BASED_REGEX);
+        if (useByteBasedRegex) {
+            return new ByteBasedRegexpSplitFunction(children);
+        } else {
+            return new StringBasedRegexpSplitFunction(children);
+        }
+    }
+}