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

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

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LikeDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LikeDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LikeDescriptor.java
deleted file mode 100644
index 2ab66fc..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/LikeDescriptor.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.evaluators.functions;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.util.Arrays;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.asterix.common.utils.UTF8CharSequence;
-import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-
-/**
- * Creates new Matcher and Pattern objects each time the value of the pattern
- * argument (the second argument) changes.
- */
-
-public class LikeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    // allowed input types
-    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
-    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new LikeDescriptor();
-        }
-    };
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.LIKE;
-    }
-
-    @Override
-    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
-
-        return new ICopyEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
-
-                final DataOutput dout = output.getDataOutput();
-
-                return new ICopyEvaluator() {
-
-                    private boolean first = true;
-                    private ArrayBackedValueStorage array0 = new ArrayBackedValueStorage();
-                    private ICopyEvaluator evalString = args[0].createEvaluator(array0);
-                    private ICopyEvaluator evalPattern = args[1].createEvaluator(array0);
-                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
-                    private UTF8CharSequence carSeq = new UTF8CharSequence();
-                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
-                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ASTRING);
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<ABoolean> booleanSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ABOOLEAN);
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ANULL);
-                    private Matcher matcher;
-                    private Pattern pattern;
-
-                    @Override
-                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
-                        // evaluate the pattern first
-                        try {
-                            array0.reset();
-                            evalPattern.evaluate(tuple);
-                            if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG) {
-                                nullSerde.serialize(ANull.NULL, dout);
-                                return;
-                            }
-                            if (array0.getByteArray()[0] != SER_STRING_TYPE_TAG) {
-                                throw new AlgebricksException(AsterixBuiltinFunctions.LIKE
-                                        + ": expects input type STRING/NULL for the first argument but got "
-                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(array0.getByteArray()[0])
-                                        + ".");
-                            }
-                            boolean newPattern = false;
-                            if (first) {
-                                first = false;
-                                newPattern = true;
-                            } else {
-                                int c = strComp.compare(array0.getByteArray(), array0.getStartOffset(),
-                                        array0.getLength(), lastPattern.getByteArray(), 0, lastPattern.size());
-                                if (c != 0) {
-                                    newPattern = true;
-                                }
-                            }
-                            if (newPattern) {
-                                lastPattern.reset();
-                                lastPattern.write(array0.getByteArray(), array0.getStartOffset(), array0.getLength());
-                                // ! object creation !
-                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
-                                        lastPattern.getByteArray()));
-                                AString strPattern = (AString) stringSerde.deserialize(di);
-                                pattern = Pattern.compile(toRegex(strPattern));
-
-                            }
-                            array0.reset();
-                            evalString.evaluate(tuple);
-                            if (array0.getByteArray()[0] == SER_NULL_TYPE_TAG) {
-                                nullSerde.serialize(ANull.NULL, dout);
-                                return;
-                            }
-                            if (array0.getByteArray()[0] != SER_STRING_TYPE_TAG) {
-                                throw new AlgebricksException(AsterixBuiltinFunctions.LIKE
-                                        + ": expects input type (STRING/NULL, STRING/NULL) but got (STRING, "
-                                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(array0.getByteArray()[0])
-                                        + ").");
-                            }
-                            carSeq.reset(array0, 1);
-                            if (newPattern) {
-                                matcher = pattern.matcher(carSeq);
-                            } else {
-                                matcher.reset(carSeq);
-                            }
-                            ABoolean res = (matcher.matches()) ? ABoolean.TRUE : ABoolean.FALSE;
-
-                            booleanSerde.serialize(res, dout);
-                        } catch (HyracksDataException e) {
-                            throw new AlgebricksException(e);
-                        }
-                    }
-
-                    private String toRegex(AString pattern) {
-                        StringBuilder sb = new StringBuilder();
-                        String str = pattern.getStringValue();
-                        for (int i = 0; i < str.length(); i++) {
-                            char c = str.charAt(i);
-                            if (c == '\\' && (i < str.length() - 1)
-                                    && (str.charAt(i + 1) == '_' || str.charAt(i + 1) == '%')) {
-                                sb.append(str.charAt(i + 1));
-                                ++i;
-                            } else if (c == '%') {
-                                sb.append(".*");
-                            } else if (c == '_') {
-                                sb.append(".");
-                            } else {
-                                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
-                                    sb.append('\\');
-                                }
-                                sb.append(c);
-                            }
-                        }
-                        return sb.toString();
-                    }
-                };
-            }
-        };
-    }
-
-    private static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$', '*', '|' };
-    static {
-        Arrays.sort(reservedRegexChars);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RegExpDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RegExpDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RegExpDescriptor.java
index 197db97..1271465 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RegExpDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RegExpDescriptor.java
@@ -24,7 +24,6 @@ import java.io.DataOutput;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.common.utils.UTF8CharSequence;
 import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.ABoolean;
@@ -45,8 +44,10 @@ import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.apache.hyracks.data.std.util.UTF8CharSequence;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 /**
@@ -92,6 +93,7 @@ public class RegExpDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                     private ICopyEvaluator evalPattern = args[1].createEvaluator(array0);
                     private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
                     private UTF8CharSequence carSeq = new UTF8CharSequence();
+                    private UTF8StringPointable utf8Ptr = new UTF8StringPointable();
                     private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
                             .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
                     @SuppressWarnings("unchecked")
@@ -153,7 +155,8 @@ public class RegExpDescriptor extends AbstractScalarFunctionDynamicDescriptor {
                                         + ": expects type STRING/NULL for the second input argument but got "
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(array0.getByteArray()[0]));
                             }
-                            carSeq.reset(array0, 1);
+                            utf8Ptr.set(array0.getByteArray(), 1, array0.getLength() - 1);
+                            carSeq.reset(utf8Ptr);
                             if (newPattern) {
                                 matcher = pattern.matcher(carSeq);
                             } else {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index ec192af..aebf186 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -84,6 +84,7 @@ public class SpatialIntersectDescriptor extends AbstractScalarFunctionDynamicDes
                     private final DoubleArray trianglesY0 = new DoubleArray();
                     private final DoubleArray trianglesX1 = new DoubleArray();
                     private final DoubleArray trianglesY1 = new DoubleArray();
+                    private final AObjectSerializerDeserializer aObjSerDer = new AObjectSerializerDeserializer();
 
                     private boolean pointOnLine(double pX, double pY, double startX, double startY, double endX,
                             double endY) throws HyracksDataException {
@@ -1095,7 +1096,7 @@ public class SpatialIntersectDescriptor extends AbstractScalarFunctionDynamicDes
                             }
 
                             ABoolean aResult = res ? (ABoolean.TRUE) : (ABoolean.FALSE);
-                            AObjectSerializerDeserializer.INSTANCE.serialize(aResult, out);
+                            aObjSerDer.serialize(aResult, out);
                         } catch (HyracksDataException hde) {
                             throw new AlgebricksException(hde);
                         }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StartsWithDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StartsWithDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StartsWithDescriptor.java
deleted file mode 100644
index f11779b..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StartsWithDescriptor.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.runtime.evaluators.functions;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-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.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-
-public class StartsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-    private static final long serialVersionUID = 1L;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new StartsWithDescriptor();
-        }
-    };
-
-    @Override
-    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
-
-        return new ICopyEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
-
-                DataOutput dout = output.getDataOutput();
-
-                return new AbstractStringContainsEval(dout, args[0], args[1], AsterixBuiltinFunctions.STARTS_WITH) {
-
-                    @Override
-                    protected boolean findMatch(byte[] strBytes, byte[] patternBytes) {
-                        int utflen1 = UTF8StringPointable.getUTFLength(strBytes, 1);
-                        int utflen2 = UTF8StringPointable.getUTFLength(patternBytes, 1);
-
-                        int s1Start = 3;
-                        int s2Start = 3;
-
-                        int c1 = 0;
-                        int c2 = 0;
-                        while (c1 < utflen1 && c2 < utflen2) {
-                            char ch1 = UTF8StringPointable.charAt(strBytes, s1Start + c1);
-                            char ch2 = UTF8StringPointable.charAt(patternBytes, s2Start + c2);
-                            if (ch1 != ch2) {
-                                break;
-                            }
-                            c1 += UTF8StringPointable.charSize(strBytes, s1Start + c1);
-                            c2 += UTF8StringPointable.charSize(patternBytes, s2Start + c2);
-                        }
-                        return (c2 == utflen2);
-                    }
-
-                };
-            }
-        };
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.STARTS_WITH;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
index 0337704..b3feba7 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringConcatDescriptor.java
@@ -38,10 +38,9 @@ import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import org.apache.hyracks.dataflow.common.data.util.StringUtils;
+import org.apache.hyracks.util.string.UTF8StringUtil;
 
 public class StringConcatDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -74,6 +73,7 @@ public class StringConcatDescriptor extends AbstractScalarFunctionDynamicDescrip
                     @SuppressWarnings("unchecked")
                     private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.ANULL);
+                    private final byte[] tempLengthArray = new byte[5];
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -81,7 +81,8 @@ public class StringConcatDescriptor extends AbstractScalarFunctionDynamicDescrip
                             outInputList.reset();
                             evalList.evaluate(tuple);
                             byte[] listBytes = outInputList.getByteArray();
-                            if (listBytes[0] != SER_ORDEREDLIST_TYPE_TAG && listBytes[0] != SER_UNORDEREDLIST_TYPE_TAG) {
+                            if (listBytes[0] != SER_ORDEREDLIST_TYPE_TAG
+                                    && listBytes[0] != SER_UNORDEREDLIST_TYPE_TAG) {
                                 throw new AlgebricksException(AsterixBuiltinFunctions.STRING_CONCAT.getName()
                                         + ": expects input type ORDEREDLIST/UNORDEREDLIST, but got "
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(listBytes[0]));
@@ -110,19 +111,20 @@ public class StringConcatDescriptor extends AbstractScalarFunctionDynamicDescrip
                                         throw new AlgebricksException(AsterixBuiltinFunctions.STRING_CONCAT.getName()
                                                 + ": expects type STRING/NULL for the list item but got " + itemType);
                                     }
-                                    utf8Len += UTF8StringPointable.getUTFLength(listBytes, itemOffset);
+                                    utf8Len += UTF8StringUtil.getUTFLength(listBytes, itemOffset);
                                 }
                                 out.writeByte(ATypeTag.STRING.serialize());
-                                StringUtils.writeUTF8Len(utf8Len, out);
+                                int cbytes = UTF8StringUtil.encodeUTF8Length(utf8Len, tempLengthArray, 0);
+                                out.write(tempLengthArray, 0, cbytes);
+
                                 for (int i = 0; i < listAccessor.size(); i++) {
                                     int itemOffset = listAccessor.getItemOffset(i);
                                     if (listAccessor.itemsAreSelfDescribing()) {
                                         itemOffset += 1;
                                     }
-                                    utf8Len = UTF8StringPointable.getUTFLength(listBytes, itemOffset);
-                                    for (int j = 0; j < utf8Len; j++) {
-                                        out.writeByte(listBytes[2 + itemOffset + j]);
-                                    }
+                                    utf8Len = UTF8StringUtil.getUTFLength(listBytes, itemOffset);
+                                    out.write(listBytes, UTF8StringUtil.getNumBytesToStoreLength(utf8Len) + itemOffset,
+                                            utf8Len);
                                 }
                             } catch (AsterixException ex) {
                                 throw new AlgebricksException(ex);

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
new file mode 100644
index 0000000..2825630
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringContainsDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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 java.io.DataOutput;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+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.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import org.apache.hyracks.data.std.api.IDataOutputProvider;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class StringContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringContainsDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1],
+                        AsterixBuiltinFunctions.STRING_CONTAINS) {
+                    @Override
+                    protected boolean compute(UTF8StringPointable left, UTF8StringPointable right)
+                            throws AlgebricksException {
+                        return UTF8StringPointable.contains(left, right, false);
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.STRING_CONTAINS;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
deleted file mode 100644
index 4c281fa..0000000
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndWithDescrtiptor.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * To change this template, choose Tools | Templates
- * and open the template in the editor.
- */
-package org.apache.asterix.runtime.evaluators.functions;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-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.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-
-public class StringEndWithDescrtiptor extends AbstractScalarFunctionDynamicDescriptor {
-    private static final long serialVersionUID = 1L;
-
-    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
-        public IFunctionDescriptor createFunctionDescriptor() {
-            return new StringEndWithDescrtiptor();
-        }
-    };
-
-    @Override
-    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
-
-        return new ICopyEvaluatorFactory() {
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
-
-                DataOutput dout = output.getDataOutput();
-
-                return new AbstractBinaryStringBoolEval(dout, args[0], args[1], AsterixBuiltinFunctions.STRING_END_WITH) {
-
-                    @Override
-                    protected boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
-                            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) {
-                        int len1 = UTF8StringPointable.getUTFLength(lBytes, 1);
-                        int len2 = UTF8StringPointable.getUTFLength(rBytes, 1);
-                        if (len2 > len1)
-                            return false;
-
-                        int pos = 3;
-                        int delta = len1 - len2;
-                        while (pos < len2 + 3) {
-                            char c1 = UTF8StringPointable.charAt(lBytes, pos + delta);
-                            char c2 = UTF8StringPointable.charAt(rBytes, pos);
-                            if (c1 != c2)
-                                return false;
-
-                            pos += UTF8StringPointable.charSize(lBytes, pos);
-                        }
-
-                        return true;
-                    }
-
-                };
-            }
-        };
-    }
-
-    @Override
-    public FunctionIdentifier getIdentifier() {
-        return AsterixBuiltinFunctions.STRING_END_WITH;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
new file mode 100644
index 0000000..673ba0f
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEndsWithDescriptor.java
@@ -0,0 +1,73 @@
+/*
+ * 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 java.io.DataOutput;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+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.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import org.apache.hyracks.data.std.api.IDataOutputProvider;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class StringEndsWithDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringEndsWithDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                DataOutput dout = output.getDataOutput();
+
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1],
+                        AsterixBuiltinFunctions.STRING_ENDS_WITH) {
+
+                    @Override
+                    protected boolean compute(UTF8StringPointable left, UTF8StringPointable right)
+                            throws AlgebricksException {
+                        return UTF8StringPointable.endsWith(left, right, false);
+                    }
+
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.STRING_ENDS_WITH;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
index 0df3e4d..312d28e 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEqualDescriptor.java
@@ -55,24 +55,9 @@ public class StringEqualDescriptor extends AbstractScalarFunctionDynamicDescript
                 return new AbstractBinaryStringBoolEval(dout, args[0], args[1], AsterixBuiltinFunctions.STRING_EQUAL) {
 
                     @Override
-                    protected boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
-                            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) {
-                        int len = UTF8StringPointable.getUTFLength(lBytes, 1);
-
-                        if (len != UTF8StringPointable.getUTFLength(rBytes, 1))
-                            return false;
-
-                        int pos = 3;
-                        while (pos < len + 3) {
-                            char c1 = UTF8StringPointable.charAt(lBytes, pos);
-                            char c2 = UTF8StringPointable.charAt(rBytes, pos);
-                            if (c1 != c2)
-                                return false;
-
-                            pos += UTF8StringPointable.charSize(lBytes, pos);
-                        }
-
-                        return true;
+                    protected boolean compute(UTF8StringPointable left, UTF8StringPointable right)
+                            throws AlgebricksException {
+                        return left.compareTo(right) == 0;
                     }
 
                 };

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
index 2d364c2..7abc721 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
@@ -18,17 +18,18 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
+import java.util.Arrays;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.om.base.AString;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
 public class StringEvaluatorUtils {
 
-    public static int toFlag(AString pattern) {
-        String str = pattern.getStringValue();
+    public static int toFlag(String pattern) {
         int flag = 0;
-        for (int i = 0; i < str.length(); i++) {
-            char c = str.charAt(i);
+        for (int i = 0; i < pattern.length(); i++) {
+            char c = pattern.charAt(i);
             switch (c) {
                 case 's':
                     flag |= Pattern.DOTALL;
@@ -47,7 +48,40 @@ public class StringEvaluatorUtils {
         return flag;
     }
 
-    public final static char[] reservedRegexChars = new char[] { '$', '(', ')', '*', '.', '[', '\\', ']', '^', '{',
-            '|', '}' };
+    public static UTF8StringPointable copyResetUTF8Pointable(UTF8StringPointable srcString,
+            ByteArrayAccessibleOutputStream destCopyStorage, UTF8StringPointable destString) {
+        destCopyStorage.reset();
+        destCopyStorage.write(srcString.getByteArray(), srcString.getStartOffset(),
+                srcString.getMetaDataLength() + srcString.getUTF8Length());
+        destString.set(destCopyStorage.getByteArray(), 0, destCopyStorage.size());
+        return destString;
+    }
+
+    static char[] reservedRegexChars = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$', '*', '|' };
+
+    static {
+        Arrays.sort(reservedRegexChars);
+    }
 
+    public static String toRegex(String pattern) {
+        StringBuilder sb = new StringBuilder();
+        for (int i = 0; i < pattern.length(); i++) {
+            char c = pattern.charAt(i);
+            if (c == '\\' && (i < pattern.length() - 1)
+                    && (pattern.charAt(i + 1) == '_' || pattern.charAt(i + 1) == '%')) {
+                sb.append(pattern.charAt(i + 1));
+                ++i;
+            } else if (c == '%') {
+                sb.append(".*");
+            } else if (c == '_') {
+                sb.append(".");
+            } else {
+                if (Arrays.binarySearch(reservedRegexChars, c) >= 0) {
+                    sb.append('\\');
+                }
+                sb.append(c);
+            }
+        }
+        return sb.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
index 31e61ff..650beb0 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringJoinDescriptor.java
@@ -34,10 +34,9 @@ import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import org.apache.hyracks.dataflow.common.data.util.StringUtils;
+import org.apache.hyracks.util.string.UTF8StringUtil;
 
 public class StringJoinDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -68,6 +67,7 @@ public class StringJoinDescriptor extends AbstractScalarFunctionDynamicDescripto
                     private ArrayBackedValueStorage outInputSep = new ArrayBackedValueStorage();
                     private ICopyEvaluator evalList = listEvalFactory.createEvaluator(outInputList);
                     private ICopyEvaluator evalSep = sepEvalFactory.createEvaluator(outInputSep);
+                    private final byte[] tempLengthArray = new byte[5];
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -96,31 +96,32 @@ public class StringJoinDescriptor extends AbstractScalarFunctionDynamicDescripto
                             try {
                                 // calculate length first
                                 int utf_8_len = 0;
-                                int sep_len = UTF8StringPointable.getUTFLength(serSep, 1);
+                                int sep_len = UTF8StringUtil.getUTFLength(serSep, 1);
+                                int sep_meta_len = UTF8StringUtil.getNumBytesToStoreLength(sep_len);
 
                                 for (int i = 0; i < size; i++) {
                                     int itemOffset = AOrderedListSerializerDeserializer
                                             .getItemOffset(serOrderedList, i);
 
-                                    int currentSize = UTF8StringPointable.getUTFLength(serOrderedList, itemOffset);
+                                    int currentSize = UTF8StringUtil.getUTFLength(serOrderedList, itemOffset);
                                     if (i != size - 1 && currentSize != 0) {
                                         utf_8_len += sep_len;
                                     }
                                     utf_8_len += currentSize;
                                 }
                                 out.writeByte(stringTypeTag);
-                                StringUtils.writeUTF8Len(utf_8_len, out);
+                                int length = UTF8StringUtil.encodeUTF8Length(utf_8_len, tempLengthArray, 0);
+                                out.write(tempLengthArray, 0, length);
                                 for (int i = 0; i < size; i++) {
                                     int itemOffset = AOrderedListSerializerDeserializer
                                             .getItemOffset(serOrderedList, i);
-                                    utf_8_len = UTF8StringPointable.getUTFLength(serOrderedList, itemOffset);
-                                    for (int j = 0; j < utf_8_len; j++) {
-                                        out.writeByte(serOrderedList[2 + itemOffset + j]);
-                                    }
+                                    utf_8_len = UTF8StringUtil.getUTFLength(serOrderedList, itemOffset);
+                                    out.write(serOrderedList,
+                                            itemOffset + UTF8StringUtil.getNumBytesToStoreLength(utf_8_len), utf_8_len);
                                     if (i == size - 1 || utf_8_len == 0)
                                         continue;
                                     for (int j = 0; j < sep_len; j++) {
-                                        out.writeByte(serSep[3 + j]);
+                                        out.writeByte(serSep[1 + sep_meta_len + j]);
                                     }
                                 }
                             } catch (AsterixException ex) {

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
index 68c5af9..4f53a53 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLengthDescriptor.java
@@ -41,6 +41,7 @@ import org.apache.hyracks.data.std.api.IDataOutputProvider;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.string.UTF8StringUtil;
 
 public class StringLengthDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -80,7 +81,7 @@ public class StringLengthDescriptor extends AbstractScalarFunctionDynamicDescrip
                             eval.evaluate(tuple);
                             byte[] serString = outInput.getByteArray();
                             if (serString[0] == SER_STRING_TYPE_TAG) {
-                                int len = UTF8StringPointable.getUTFLength(outInput.getByteArray(), 1);
+                                int len = UTF8StringUtil.getUTFLength(outInput.getByteArray(), 1);
                                 result.setValue((long) len);
                                 int64Serde.serialize(result, out);
                             } else if (serString[0] == SER_NULL_TYPE_TAG)

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
new file mode 100644
index 0000000..3ccba1c
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLikeDescriptor.java
@@ -0,0 +1,106 @@
+/*
+ * 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 java.io.DataOutput;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+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.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import org.apache.hyracks.data.std.api.IDataOutputProvider;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.apache.hyracks.data.std.util.UTF8CharSequence;
+
+/**
+ * Creates new Matcher and Pattern objects each time the value of the pattern
+ * argument (the second argument) changes.
+ */
+
+public class StringLikeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new StringLikeDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.STRING_LIKE;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
+
+                final DataOutput dout = output.getDataOutput();
+
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1],
+                        AsterixBuiltinFunctions.STRING_MATCHES) {
+
+                    private Pattern pattern = null;
+                    private Matcher matcher = null;
+                    private ByteArrayAccessibleOutputStream lastPatternStorage = new ByteArrayAccessibleOutputStream();
+                    private UTF8StringPointable lastPatternPtr = new UTF8StringPointable();
+                    private UTF8CharSequence carSeq = new UTF8CharSequence();
+
+                    @Override
+                    protected boolean compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr)
+                            throws AlgebricksException {
+                        boolean newPattern = false;
+                        if (pattern == null || lastPatternPtr.compareTo(patternPtr) != 0) {
+                            newPattern = true;
+                        }
+                        if (newPattern) {
+                            StringEvaluatorUtils.copyResetUTF8Pointable(patternPtr, lastPatternStorage, lastPatternPtr);
+                            // ! object creation !
+                            pattern = Pattern.compile(StringEvaluatorUtils.toRegex(lastPatternPtr.toString()));
+                        }
+
+                        carSeq.reset(srcPtr);
+                        if (newPattern) {
+                            matcher = pattern.matcher(carSeq);
+                        } else {
+                            matcher.reset(carSeq);
+                        }
+                        return matcher.find();
+                    }
+
+                };
+            }
+        };
+    }
+
+};
+

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
index 0d4148b..66212f7 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringLowerCaseDescriptor.java
@@ -38,8 +38,9 @@ import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
 import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.GrowableArray;
+import org.apache.hyracks.data.std.util.UTF8StringBuilder;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import org.apache.hyracks.dataflow.common.data.util.StringUtils;
 
 public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
@@ -61,9 +62,13 @@ public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDesc
             public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
                 return new ICopyEvaluator() {
 
-                    private DataOutput out = output.getDataOutput();
-                    private ArrayBackedValueStorage outInput = new ArrayBackedValueStorage();
-                    private ICopyEvaluator eval = args[0].createEvaluator(outInput);
+                    private final DataOutput out = output.getDataOutput();
+                    private final ArrayBackedValueStorage outInput = new ArrayBackedValueStorage();
+                    private final ICopyEvaluator eval = args[0].createEvaluator(outInput);
+
+                    private final GrowableArray array = new GrowableArray();
+                    private final UTF8StringBuilder builder = new UTF8StringBuilder();
+                    private final UTF8StringPointable string = new UTF8StringPointable();
 
                     private final byte stt = ATypeTag.STRING.serialize();
 
@@ -80,19 +85,12 @@ public class StringLowerCaseDescriptor extends AbstractScalarFunctionDynamicDesc
                             byte[] serString = outInput.getByteArray();
 
                             if (serString[0] == SER_STRING_TYPE_TAG) {
-                                byte[] bytes = outInput.getByteArray();
-                                int len = UTF8StringPointable.getUTFLength(bytes, 1);
+                                string.set(serString, 1, serString.length);
+                                array.reset();
+                                UTF8StringPointable.lowercase(string, builder, array);
 
                                 out.writeByte(stt);
-                                StringUtils.writeUTF8Len(len, out);
-
-                                int pos = 3;
-                                while (pos < len + 3) {
-                                    char c1 = UTF8StringPointable.charAt(bytes, pos);
-                                    c1 = Character.toLowerCase(c1);
-                                    pos += UTF8StringPointable.charSize(bytes, pos);
-                                    StringUtils.writeCharAsModifiedUTF8(c1, out);
-                                }
+                                out.write(array.getByteArray(), 0, array.getLength());
                             } else if (serString[0] == SER_NULL_TYPE_TAG)
                                 nullSerde.serialize(ANull.NULL, out);
                             else

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
index 43a6d59..0b8477c 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesDescriptor.java
@@ -18,31 +18,22 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.common.utils.UTF8CharSequence;
-import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.apache.hyracks.data.std.util.UTF8CharSequence;
 
 public class StringMatchesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
     private static final long serialVersionUID = 1L;
@@ -64,54 +55,35 @@ public class StringMatchesDescriptor extends AbstractScalarFunctionDynamicDescri
 
                 DataOutput dout = output.getDataOutput();
 
-                return new AbstractBinaryStringBoolEval(dout, args[0], args[1], AsterixBuiltinFunctions.STRING_MATCHES) {
+                return new AbstractBinaryStringBoolEval(dout, args[0], args[1],
+                        AsterixBuiltinFunctions.STRING_MATCHES) {
 
                     private Pattern pattern = null;
                     private Matcher matcher = null;
-                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
-                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
-                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
+                    private ByteArrayAccessibleOutputStream lastPatternStorage = new ByteArrayAccessibleOutputStream();
+                    private UTF8StringPointable lastPatternPtr = new UTF8StringPointable();
                     private UTF8CharSequence carSeq = new UTF8CharSequence();
 
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ASTRING);
-
                     @Override
-                    protected boolean compute(byte[] lBytes, int lLen, int lStart, byte[] rBytes, int rLen, int rStart,
-                            ArrayBackedValueStorage array0, ArrayBackedValueStorage array1) throws AlgebricksException {
-                        try {
-                            boolean newPattern = false;
-                            if (pattern == null) {
-                                newPattern = true;
-                            } else {
-                                int c = strComp.compare(rBytes, rStart, rLen, lastPattern.getByteArray(), 0,
-                                        lastPattern.size());
-                                if (c != 0) {
-                                    newPattern = true;
-                                }
-                            }
-                            if (newPattern) {
-                                lastPattern.reset();
-                                lastPattern.write(rBytes, rStart, rLen);
-                                // ! object creation !
-                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
-                                        lastPattern.getByteArray()));
-                                AString strPattern = (AString) stringSerde.deserialize(di);
-                                // pattern = Pattern.compile(toRegex(strPattern));
-                                pattern = Pattern.compile(strPattern.getStringValue());
-                            }
+                    protected boolean compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr)
+                            throws AlgebricksException {
+                        boolean newPattern = false;
+                        if (pattern == null || lastPatternPtr.compareTo(patternPtr) != 0) {
+                            newPattern = true;
+                        }
+                        if (newPattern) {
+                            StringEvaluatorUtils.copyResetUTF8Pointable(patternPtr, lastPatternStorage, lastPatternPtr);
+                            // ! object creation !
+                            pattern = Pattern.compile(lastPatternPtr.toString());
+                        }
 
-                            carSeq.reset(array0, 1);
-                            if (newPattern) {
-                                matcher = pattern.matcher(carSeq);
-                            } else {
-                                matcher.reset(carSeq);
-                            }
-                            return matcher.find();
-                        } catch (HyracksDataException e) {
-                            throw new AlgebricksException(e);
+                        carSeq.reset(srcPtr);
+                        if (newPattern) {
+                            matcher = pattern.matcher(carSeq);
+                        } else {
+                            matcher.reset(carSeq);
                         }
+                        return matcher.find();
                     }
 
                 };

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
index 74089b1..e4129a3 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringMatchesWithFlagDescriptor.java
@@ -22,31 +22,22 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.common.utils.UTF8CharSequence;
-import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.apache.hyracks.data.std.util.UTF8CharSequence;
 
 public class StringMatchesWithFlagDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -71,76 +62,39 @@ public class StringMatchesWithFlagDescriptor extends AbstractScalarFunctionDynam
 
                 return new AbstractTripleStringBoolEval(dout, args[0], args[1], args[2],
                         AsterixBuiltinFunctions.STRING_MATCHES_WITH_FLAG) {
-
                     private Pattern pattern = null;
                     private Matcher matcher = null;
-                    private String strPattern = "";
-                    private int flags = 0;
-                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
-                    private ByteArrayAccessibleOutputStream lastFlags = new ByteArrayAccessibleOutputStream();
-                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
-                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
+                    private ByteArrayAccessibleOutputStream lastPatternStorage = new ByteArrayAccessibleOutputStream();
+                    private ByteArrayAccessibleOutputStream lastFlagsStorage = new ByteArrayAccessibleOutputStream();
+                    private UTF8StringPointable lastPatternPtr = new UTF8StringPointable();
+                    private UTF8StringPointable lastFlagPtr = new UTF8StringPointable();
                     private UTF8CharSequence carSeq = new UTF8CharSequence();
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ASTRING);
 
                     @Override
-                    protected boolean compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2,
-                            int s2, ArrayBackedValueStorage array0, ArrayBackedValueStorage array1)
-                            throws AlgebricksException {
-                        try {
-                            boolean newPattern = false;
-                            boolean newFlags = false;
-
-                            AString astrPattern;
-                            AString astrFlags;
+                    protected boolean compute(UTF8StringPointable strSrc, UTF8StringPointable strPattern,
+                            UTF8StringPointable strFlag) throws AlgebricksException {
+                        final boolean newPattern = (pattern == null || lastPatternPtr.compareTo(strPattern) != 0);
+                        final boolean newFlag = (pattern == null || lastFlagPtr.compareTo(strFlag) != 0);
 
-                            if (pattern == null) {
-                                newPattern = true;
-                                newFlags = true;
-                            } else {
-                                int c = strComp.compare(b1, s1, l1, lastPattern.getByteArray(), 0, lastPattern.size());
-                                if (c != 0) {
-                                    newPattern = true;
-                                }
+                        if (newPattern) {
+                            StringEvaluatorUtils.copyResetUTF8Pointable(strPattern, lastPatternStorage, lastPatternPtr);
+                        }
 
-                                c = strComp.compare(b2, s2, l2, lastFlags.getByteArray(), 0, lastFlags.size());
-                                if (c != 0) {
-                                    newFlags = true;
-                                }
-                            }
-                            if (newPattern) {
-                                lastPattern.reset();
-                                lastPattern.write(b1, s1, l1);
-                                // ! object creation !
-                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
-                                        lastPattern.getByteArray()));
-                                astrPattern = (AString) stringSerde.deserialize(di);
-                                // strPattern = toRegex(astrPattern);
-                                strPattern = astrPattern.getStringValue();
-                            }
-                            if (newFlags) {
-                                lastFlags.reset();
-                                lastFlags.write(b2, s2, l2);
-                                // ! object creation !
-                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
-                                        lastFlags.getByteArray()));
-                                astrFlags = (AString) stringSerde.deserialize(di);
-                                flags = StringEvaluatorUtils.toFlag(astrFlags);
-                            }
+                        if (newFlag) {
+                            StringEvaluatorUtils.copyResetUTF8Pointable(strFlag, lastFlagsStorage, lastFlagPtr);
+                        }
 
-                            pattern = Pattern.compile(strPattern, flags);
-                            carSeq.reset(array0, 1);
-                            if (newPattern) {
-                                matcher = pattern.matcher(carSeq);
-                            } else {
-                                matcher.reset(carSeq);
-                            }
-                            return matcher.find();
-                        } catch (HyracksDataException e) {
-                            throw new AlgebricksException(e);
+                        if (newPattern || newFlag) {
+                            pattern = Pattern.compile(lastPatternPtr.toString(),
+                                    StringEvaluatorUtils.toFlag(lastFlagPtr.toString()));
+                        }
+                        carSeq.reset(strSrc);
+                        if (newPattern || newFlag) {
+                            matcher = pattern.matcher(carSeq);
+                        } else {
+                            matcher.reset(carSeq);
                         }
+                        return matcher.find();
                     }
                 };
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/742aba85/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
----------------------------------------------------------------------
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
index e0628e2..a43f1a4 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReplaceDescriptor.java
@@ -18,31 +18,22 @@
  */
 package org.apache.asterix.runtime.evaluators.functions;
 
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.common.utils.UTF8CharSequence;
-import org.apache.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import org.apache.asterix.om.base.AString;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import org.apache.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.api.IDataOutputProvider;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import org.apache.hyracks.data.std.util.UTF8CharSequence;
 
 public class StringReplaceDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
@@ -70,79 +61,42 @@ public class StringReplaceDescriptor extends AbstractScalarFunctionDynamicDescri
 
                     private Pattern pattern = null;
                     private Matcher matcher = null;
-                    private String replace;
-                    private String strPattern = "";
+                    private String replaceStr;
                     private StringBuffer resultBuf = new StringBuffer();
-                    private ByteArrayAccessibleOutputStream lastPattern = new ByteArrayAccessibleOutputStream();
-                    private ByteArrayAccessibleOutputStream lastReplace = new ByteArrayAccessibleOutputStream();
-                    private IBinaryComparator strComp = AqlBinaryComparatorFactoryProvider.INSTANCE
-                            .getBinaryComparatorFactory(BuiltinType.ASTRING, true).createBinaryComparator();
+                    private ByteArrayAccessibleOutputStream lastPatternStorage = new ByteArrayAccessibleOutputStream();
+                    private UTF8StringPointable lastPatternPtr = new UTF8StringPointable();
+                    private ByteArrayAccessibleOutputStream lastReplaceStorage = new ByteArrayAccessibleOutputStream();
+                    private UTF8StringPointable lastReplacePtr = new UTF8StringPointable();
                     private UTF8CharSequence carSeq = new UTF8CharSequence();
-                    @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.ASTRING);
 
                     @Override
-                    protected String compute(byte[] b0, int l0, int s0, byte[] b1, int l1, int s1, byte[] b2, int l2,
-                            int s2, ArrayBackedValueStorage array0, ArrayBackedValueStorage array1)
-                            throws AlgebricksException {
-                        try {
-                            boolean newPattern = false;
-                            boolean newReplace = false;
-
-                            AString astrPattern;
-                            AString astrReplace;
+                    protected String compute(UTF8StringPointable srcPtr, UTF8StringPointable patternPtr,
+                            UTF8StringPointable replacePtr) throws AlgebricksException {
+                        resultBuf.setLength(0);
+                        final boolean newPattern = (pattern == null || lastPatternPtr.compareTo(patternPtr) != 0);
+                        final boolean newReplace = (pattern == null || lastReplacePtr.compareTo(replacePtr) != 0);
+                        if (newPattern) {
+                            StringEvaluatorUtils.copyResetUTF8Pointable(patternPtr, lastPatternStorage, lastPatternPtr);
+                            // ! object creation !
+                            pattern = Pattern.compile(lastPatternPtr.toString());
+                        }
+                        if (newReplace) {
+                            StringEvaluatorUtils.copyResetUTF8Pointable(replacePtr, lastReplaceStorage, lastReplacePtr);
+                            replaceStr = replacePtr.toString();
+                        }
 
-                            if (pattern == null) {
-                                newPattern = true;
-                                newReplace = true;
-                            } else {
-                                int c = strComp.compare(b1, s1, l1, lastPattern.getByteArray(), 0, lastPattern.size());
-                                if (c != 0) {
-                                    newPattern = true;
-                                }
+                        carSeq.reset(srcPtr);
+                        if (newPattern) {
+                            matcher = pattern.matcher(carSeq);
+                        } else {
+                            matcher.reset(carSeq);
+                        }
 
-                                c = strComp.compare(b2, s2, l2, lastReplace.getByteArray(), 0, lastReplace.size());
-                                if (c != 0) {
-                                    newReplace = true;
-                                }
-                            }
-                            if (newPattern) {
-                                lastPattern.reset();
-                                lastPattern.write(b1, s1, l1);
-                                // ! object creation !
-                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
-                                        lastPattern.getByteArray()));
-                                astrPattern = (AString) stringSerde.deserialize(di);
-                                // strPattern = toRegex(astrPattern);
-                                strPattern = astrPattern.getStringValue();
-                            }
-                            if (newReplace) {
-                                lastReplace.reset();
-                                lastReplace.write(b2, s2, l2);
-                                // ! object creation !
-                                DataInputStream di = new DataInputStream(new ByteArrayInputStream(
-                                        lastReplace.getByteArray()));
-                                astrReplace = (AString) stringSerde.deserialize(di);
-                                replace = astrReplace.getStringValue();
-                            }
-                            if (newPattern) {
-                                pattern = Pattern.compile(strPattern);
-                            }
-                            carSeq.reset(array0, 1);
-                            if (newPattern) {
-                                matcher = pattern.matcher(carSeq);
-                            } else {
-                                matcher.reset(carSeq);
-                            }
-                            while (matcher.find()) {
-                                matcher.appendReplacement(resultBuf, replace);
-                            }
-                            matcher.appendTail(resultBuf);
-                            return resultBuf.toString();
-                        } catch (HyracksDataException e) {
-                            throw new AlgebricksException(e);
+                        while (matcher.find()) {
+                            matcher.appendReplacement(resultBuf, replaceStr);
                         }
+                        matcher.appendTail(resultBuf);
+                        return resultBuf.toString();
                     }
                 };
             }