You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/09/21 17:41:49 UTC

svn commit: r1525253 - in /hive/branches/vectorization/ql/src: java/org/apache/hadoop/hive/ql/exec/vector/ java/org/apache/hadoop/hive/ql/exec/vector/expressions/ java/org/apache/hadoop/hive/ql/optimizer/physical/ test/org/apache/hadoop/hive/ql/exec/ve...

Author: hashutosh
Date: Sat Sep 21 15:41:49 2013
New Revision: 1525253

URL: http://svn.apache.org/r1525253
Log:
HIVE-4624 : Integrate Vectorized Substr into Vectorized QE (Eric Hanson via Ashutosh Chauhan)

Modified:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java?rev=1525253&r1=1525252&r2=1525253&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java Sat Sep 21 15:41:49 2013
@@ -46,6 +46,8 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringConcatColCol;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringConcatColScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringConcatScalarCol;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringSubstrColStart;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringSubstrColStartLen;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorUDFUnixTimeStampLong;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
@@ -93,6 +95,7 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.UDFOPPlus;
 import org.apache.hadoop.hive.ql.udf.UDFOPPositive;
 import org.apache.hadoop.hive.ql.udf.UDFSecond;
+import org.apache.hadoop.hive.ql.udf.UDFSubstr;
 import org.apache.hadoop.hive.ql.udf.UDFWeekOfYear;
 import org.apache.hadoop.hive.ql.udf.UDFYear;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
@@ -509,6 +512,8 @@ public class VectorizationContext {
       return getLikeExpression(childExpr);
     } else if (cl.equals(UDFLength.class)) {
       return getUnaryStringExpression("StringLength", "Long", childExpr);
+    } else if (cl.equals(UDFSubstr.class)) {
+      return getSubstrExpression(childExpr);
     }
 
     throw new HiveException("Udf: "+udf.getClass().getSimpleName()+", is not supported");
@@ -739,6 +744,65 @@ public class VectorizationContext {
     return expr;
   }
 
+  private VectorExpression getSubstrExpression(
+      List<ExprNodeDesc> childExprList) throws HiveException {
+
+    ExprNodeDesc childExpr = childExprList.get(0);
+    ExprNodeDesc startExpr = childExprList.get(1);
+    startExpr = foldConstantsForUnaryExpression(startExpr);
+
+    // Get second and optionally third arguments
+    int start;
+    if (startExpr instanceof ExprNodeConstantDesc) {
+      ExprNodeConstantDesc constDesc = (ExprNodeConstantDesc) startExpr;
+      start = ((Integer) constDesc.getValue()).intValue();
+    } else {
+      throw new HiveException("Cannot vectorize non-constant start argument for SUBSTR");
+    }
+    ExprNodeDesc lengthExpr = null;
+    int length = 0;
+    if (childExprList.size() == 3) {
+      lengthExpr = childExprList.get(2);
+      lengthExpr = foldConstantsForUnaryExpression(lengthExpr);
+      if (lengthExpr instanceof ExprNodeConstantDesc) {
+        ExprNodeConstantDesc constDesc = (ExprNodeConstantDesc) lengthExpr;
+        length = ((Integer) constDesc.getValue()).intValue();
+      } else {
+        throw new HiveException("Cannot vectorize non-constant length argument for SUBSTR");
+      }
+    }
+
+    // Prepare first argument (whether it is a column or an expression)
+    int inputCol;
+    VectorExpression v1 = null;
+    if (childExpr instanceof ExprNodeGenericFuncDesc) {
+      v1 = getVectorExpression(childExpr);
+      inputCol = v1.getOutputColumn();
+    } else if (childExpr instanceof ExprNodeColumnDesc) {
+      ExprNodeColumnDesc colDesc = (ExprNodeColumnDesc) childExpr;
+      inputCol = getInputColumnIndex(colDesc.getColumn());
+    } else {
+      throw new HiveException("Expression not supported: " + childExpr);
+    }
+    int outputCol = ocm.allocateOutputColumn("String");
+
+    // Create appropriate vector expression for 2 or 3 argument version of SUBSTR()
+    VectorExpression expr = null;
+    if (childExprList.size() == 2) {
+      expr = new StringSubstrColStart(inputCol, start, outputCol);
+    } else if (childExprList.size() == 3) {
+      expr = new StringSubstrColStartLen(inputCol, start, length, outputCol);
+    } else {
+      throw new HiveException("Invalid number of arguments for SUBSTR()");
+    }
+
+    if (v1 != null) {
+      expr.setChildExpressions(new VectorExpression [] {v1});
+      ocm.freeOutputColumn(v1.getOutputColumn());
+    }
+    return expr;
+  }
+
   private VectorExpression getLikeExpression(List<ExprNodeDesc> childExpr) throws HiveException {
     ExprNodeDesc leftExpr = childExpr.get(0);
     ExprNodeDesc rightExpr = childExpr.get(1);

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java?rev=1525253&r1=1525252&r2=1525253&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStart.java Sat Sep 21 15:41:49 2013
@@ -33,13 +33,13 @@ public class StringSubstrColStart extend
   private int startIdx;
   private int colNum;
   private int outputColumn;
-  private transient static byte[] EMPTYSTRING;
+  private transient static byte[] EMPTY_STRING;
 
   // Populating the Empty string bytes. Putting it as static since it should be immutable and can
   // be shared.
   static {
     try {
-      EMPTYSTRING = "".getBytes("UTF-8");
+      EMPTY_STRING = "".getBytes("UTF-8");
     } catch(UnsupportedEncodingException e) {
       e.printStackTrace();
     }
@@ -48,7 +48,22 @@ public class StringSubstrColStart extend
   public StringSubstrColStart(int colNum, int startIdx, int outputColumn) {
     this();
     this.colNum = colNum;
-    this.startIdx = startIdx;
+
+    /* Switch from a 1-based start offset (the Hive end user convention) to a 0-based start offset
+     * (the internal convention).
+     */
+    if (startIdx >= 1) {
+      this.startIdx = startIdx - 1;
+    } else if (startIdx == 0) {
+
+      // If start index is 0 in query, that is equivalent to using 1 in query.
+      // So internal offset is 0.
+      this.startIdx = 0;
+    } else {
+
+      // start index of -n means give the last n characters of the string
+      this.startIdx = startIdx;
+    }
     this.outputColumn = outputColumn;
   }
 
@@ -65,24 +80,27 @@ public class StringSubstrColStart extend
    * @param substrStart the Start index for the substring operation
    */
   static int getSubstrStartOffset(byte[] utf8String, int start, int len, int substrStart) {
-    int curIdx = -1;
+    int end = start + len;
 
     if (substrStart < 0) {
       int length = 0;
-      for (int i = start; i != len; ++i) {
+      for (int i = start; i != end; ++i) {
         if ((utf8String[i] & 0xc0) != 0x80) {
           ++length;
         }
       }
+      if (-substrStart > length) {
 
-      if (-length > substrStart) {
+        /* The result is empty string if a negative start is provided
+         * whose absolute value is greater than the string length.
+         */
         return -1;
       }
 
       substrStart = length + substrStart;
     }
 
-    int end = start + len;
+    int curIdx = -1;
     for (int i = start; i != end; ++i) {
       if ((utf8String[i] & 0xc0) != 0x80) {
         ++curIdx;
@@ -109,26 +127,26 @@ public class StringSubstrColStart extend
       return;
     }
 
-
     byte[][] vector = inV.vector;
     int[] sel = batch.selected;
     int[] len = inV.length;
     int[] start = inV.start;
+    outV.initBuffer();
 
     if (inV.isRepeating) {
       outV.isRepeating = true;
       if (!inV.noNulls && inV.isNull[0]) {
         outV.isNull[0] = true;
         outV.noNulls = false;
-        outV.setRef(0, EMPTYSTRING, 0, EMPTYSTRING.length);
+        outV.setVal(0, EMPTY_STRING, 0, EMPTY_STRING.length);
         return;
       } else {
         outV.noNulls = true;
-        int offset = getSubstrStartOffset(vector[0], sel[0], len[0], startIdx);
+        int offset = getSubstrStartOffset(vector[0], start[0], len[0], startIdx);
         if (offset != -1) {
-          outV.setRef(0, vector[0], offset, len[0] - offset);
+          outV.setVal(0, vector[0], offset, len[0] - (offset - start[0]));
         } else {
-          outV.setRef(0, EMPTYSTRING, 0, EMPTYSTRING.length);
+          outV.setVal(0, EMPTY_STRING, 0, EMPTY_STRING.length);
         }
       }
     } else {
@@ -143,9 +161,10 @@ public class StringSubstrColStart extend
                   startIdx);
               outV.isNull[selected] = false;
               if (offset != -1) {
-                outV.setRef(selected, vector[selected], offset, len[selected] - offset);
+                outV.setVal(selected, vector[selected], offset,
+                    len[selected] - (offset - start[selected]));
               } else {
-                outV.setRef(selected, EMPTYSTRING, 0, EMPTYSTRING.length);
+                outV.setVal(selected, EMPTY_STRING, 0, EMPTY_STRING.length);
               }
             } else {
               outV.isNull[selected] = true;
@@ -158,9 +177,10 @@ public class StringSubstrColStart extend
             int offset = getSubstrStartOffset(vector[selected], start[selected], len[selected],
                 startIdx);
             if (offset != -1) {
-              outV.setRef(selected, vector[selected], offset, len[selected] - offset);
+              outV.setVal(selected, vector[selected], offset,
+                  len[selected] - (offset - start[selected]));
             } else {
-              outV.setRef(selected, EMPTYSTRING, 0, EMPTYSTRING.length);
+              outV.setVal(selected, EMPTY_STRING, 0, EMPTY_STRING.length);
             }
           }
         }
@@ -172,9 +192,9 @@ public class StringSubstrColStart extend
             if (!inV.isNull[i]) {
               int offset = getSubstrStartOffset(vector[i], start[i], len[i], startIdx);
               if (offset != -1) {
-                outV.setRef(i, vector[i], offset, len[i] - offset);
+                outV.setVal(i, vector[i], offset, len[i] - (offset - start[i]));
               } else {
-                outV.setRef(i, EMPTYSTRING, 0, EMPTYSTRING.length);
+                outV.setVal(i, EMPTY_STRING, 0, EMPTY_STRING.length);
               }
             }
           }
@@ -183,9 +203,9 @@ public class StringSubstrColStart extend
           for (int i = 0; i != n; ++i) {
             int offset = getSubstrStartOffset(vector[i], start[i], len[i], startIdx);
             if (offset != -1) {
-              outV.setRef(i, vector[i], offset, len[i] - offset);
+              outV.setVal(i, vector[i], offset, len[i] - (offset - start[i]));
             } else {
-              outV.setRef(i, EMPTYSTRING, 0, EMPTYSTRING.length);
+              outV.setVal(i, EMPTY_STRING, 0, EMPTY_STRING.length);
             }
           }
         }

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java?rev=1525253&r1=1525252&r2=1525253&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StringSubstrColStartLen.java Sat Sep 21 15:41:49 2013
@@ -37,13 +37,13 @@ public class StringSubstrColStartLen ext
   private int length;
   private int outputColumn;
   private transient final int[] offsetArray;
-  private transient static byte[] EMPTYSTRING;
+  private transient static byte[] EMPTY_STRING;
 
   // Populating the Empty string bytes. Putting it as static since it should be immutable and can be
   // shared
   static {
     try {
-      EMPTYSTRING = "".getBytes("UTF-8");
+      EMPTY_STRING = "".getBytes("UTF-8");
     } catch(UnsupportedEncodingException e) {
       e.printStackTrace();
     }
@@ -52,7 +52,23 @@ public class StringSubstrColStartLen ext
   public StringSubstrColStartLen(int colNum, int startIdx, int length, int outputColumn) {
     this();
     this.colNum = colNum;
-    this.startIdx = startIdx;
+
+    /* Switch from a 1-based start offset (the Hive end user convention) to a 0-based start offset
+     * (the internal convention).
+     */
+    if (startIdx >= 1) {
+      this.startIdx = startIdx - 1;
+    } else if (startIdx == 0) {
+
+      // If start index is 0 in query, that is equivalent to using 1 in query.
+      // So internal offset is 0.
+      this.startIdx = 0;
+    } else {
+
+      // start index of -n means give the last n characters of the string
+      this.startIdx = startIdx;
+    }
+
     this.length = length;
     this.outputColumn = outputColumn;
   }
@@ -77,25 +93,24 @@ public class StringSubstrColStartLen ext
     int curIdx = -1;
     offsetArray[0] = -1;
     offsetArray[1] = -1;
+    int end = start + len;
 
     if (substrStart < 0) {
       int length = 0;
-      for (int i = start; i != len; ++i) {
+      for (int i = start; i != end; ++i) {
         if ((utf8String[i] & 0xc0) != 0x80) {
           ++length;
         }
       }
 
-      if (-length > substrStart) {
+      if (-substrStart > length) {
         return;
       }
 
       substrStart = length + substrStart;
     }
 
-
     int endIdx = substrStart + substrLength - 1;
-    int end = start + len;
     for (int i = start; i != end; ++i) {
       if ((utf8String[i] & 0xc0) != 0x80) {
         ++curIdx;
@@ -131,21 +146,22 @@ public class StringSubstrColStartLen ext
     int[] sel = batch.selected;
     int[] len = inV.length;
     int[] start = inV.start;
+    outV.initBuffer();
 
     if (inV.isRepeating) {
       outV.isRepeating = true;
       if (!inV.noNulls && inV.isNull[0]) {
         outV.isNull[0] = true;
         outV.noNulls = false;
-        outV.setRef(0, EMPTYSTRING, 0, EMPTYSTRING.length);
+        outV.setVal(0, EMPTY_STRING, 0, EMPTY_STRING.length);
         return;
       } else {
         outV.noNulls = true;
-        populateSubstrOffsets(vector[0], sel[0], len[0], startIdx, length, offsetArray);
+        populateSubstrOffsets(vector[0], start[0], len[0], startIdx, length, offsetArray);
         if (offsetArray[0] != -1) {
-          outV.setRef(0, vector[0], offsetArray[0], offsetArray[1]);
+          outV.setVal(0, vector[0], offsetArray[0], offsetArray[1]);
         } else {
-          outV.setRef(0, EMPTYSTRING, 0, EMPTYSTRING.length);
+          outV.setVal(0, EMPTY_STRING, 0, EMPTY_STRING.length);
         }
       }
     } else {
@@ -160,9 +176,9 @@ public class StringSubstrColStartLen ext
               populateSubstrOffsets(vector[selected], start[selected], len[selected], startIdx,
                   length, offsetArray);
               if (offsetArray[0] != -1) {
-                outV.setRef(selected, vector[selected], offsetArray[0], offsetArray[1]);
+                outV.setVal(selected, vector[selected], offsetArray[0], offsetArray[1]);
               } else {
-                outV.setRef(selected, EMPTYSTRING, 0, EMPTYSTRING.length);
+                outV.setVal(selected, EMPTY_STRING, 0, EMPTY_STRING.length);
               }
             } else {
               outV.isNull[selected] = true;
@@ -176,9 +192,9 @@ public class StringSubstrColStartLen ext
             populateSubstrOffsets(vector[selected], start[selected], len[selected], startIdx,
                 length, offsetArray);
             if (offsetArray[0] != -1) {
-              outV.setRef(selected, vector[selected], offsetArray[0], offsetArray[1]);
+              outV.setVal(selected, vector[selected], offsetArray[0], offsetArray[1]);
             } else {
-              outV.setRef(selected, EMPTYSTRING, 0, EMPTYSTRING.length);
+              outV.setVal(selected, EMPTY_STRING, 0, EMPTY_STRING.length);
             }
           }
         }
@@ -190,9 +206,9 @@ public class StringSubstrColStartLen ext
             if (!inV.isNull[i]) {
               populateSubstrOffsets(vector[i], start[i], len[i], startIdx, length, offsetArray);
               if (offsetArray[0] != -1) {
-                outV.setRef(i, vector[i], offsetArray[0], offsetArray[1]);
+                outV.setVal(i, vector[i], offsetArray[0], offsetArray[1]);
               } else {
-                outV.setRef(i, EMPTYSTRING, 0, EMPTYSTRING.length);
+                outV.setVal(i, EMPTY_STRING, 0, EMPTY_STRING.length);
               }
             }
           }
@@ -202,9 +218,9 @@ public class StringSubstrColStartLen ext
             outV.isNull[i] = false;
             populateSubstrOffsets(vector[i], start[i], len[i], startIdx, length, offsetArray);
             if (offsetArray[0] != -1) {
-              outV.setRef(i, vector[i], offsetArray[0], offsetArray[1]);
+              outV.setVal(i, vector[i], offsetArray[0], offsetArray[1]);
             } else {
-              outV.setRef(i, EMPTYSTRING, 0, EMPTYSTRING.length);
+              outV.setVal(i, EMPTY_STRING, 0, EMPTY_STRING.length);
             }
           }
         }

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java?rev=1525253&r1=1525252&r2=1525253&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java Sat Sep 21 15:41:49 2013
@@ -81,6 +81,7 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.UDFOPPlus;
 import org.apache.hadoop.hive.ql.udf.UDFOPPositive;
 import org.apache.hadoop.hive.ql.udf.UDFSecond;
+import org.apache.hadoop.hive.ql.udf.UDFSubstr;
 import org.apache.hadoop.hive.ql.udf.UDFWeekOfYear;
 import org.apache.hadoop.hive.ql.udf.UDFYear;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
@@ -159,6 +160,7 @@ public class Vectorizer implements Physi
     supportedGenericUDFs.add(UDFDayOfMonth.class);
 
     supportedGenericUDFs.add(UDFLike.class);
+    supportedGenericUDFs.add(UDFSubstr.class);
     supportedGenericUDFs.add(GenericUDFLower.class);
     supportedGenericUDFs.add(GenericUDFUpper.class);
     supportedGenericUDFs.add(GenericUDFConcat.class);

Modified: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java?rev=1525253&r1=1525252&r2=1525253&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java (original)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java Sat Sep 21 15:41:49 2013
@@ -991,7 +991,7 @@ public class TestVectorStringExpressions
     v.setRef(2, data3, 0, data3.length);
     v.isNull[2] = false;
 
-    StringSubstrColStart expr = new StringSubstrColStart(0, 5, 1);
+    StringSubstrColStart expr = new StringSubstrColStart(0, 6, 1);
     expr.evaluate(batch);
     BytesColumnVector outCol = (BytesColumnVector) batch.cols[1];
     Assert.assertEquals(3, batch.size);
@@ -1021,7 +1021,7 @@ public class TestVectorStringExpressions
     outCol.isRepeating = true;
 
     // Testing negative substring index.
-    // For a string with length 11, start idx 5 should yield same results as -6
+    // Start index -6 should yield the last 6 characters of the string
 
     expr = new StringSubstrColStart(0, -6, 1);
     expr.evaluate(batch);
@@ -1050,9 +1050,9 @@ public class TestVectorStringExpressions
     outCol.noNulls = false;
     outCol.isRepeating = true;
 
-    // Testing substring starting from index 0
+    // Testing substring starting from index 1
 
-    expr = new StringSubstrColStart(0, 0, 1);
+    expr = new StringSubstrColStart(0, 1, 1);
     expr.evaluate(batch);
     Assert.assertEquals(3, batch.size);
     Assert.assertTrue(outCol.noNulls);
@@ -1081,7 +1081,7 @@ public class TestVectorStringExpressions
 
     // Testing with nulls
 
-    expr = new StringSubstrColStart(0, 5, 1);
+    expr = new StringSubstrColStart(0, 6, 1);
     v.noNulls = false;
     v.isNull[0] = true;
     expr.evaluate(batch);
@@ -1135,7 +1135,7 @@ public class TestVectorStringExpressions
     batch.cols[1] = outV;
     outV.isRepeating = true;
     outV.noNulls = false;
-    expr = new StringSubstrColStart(0, 2, 1);
+    expr = new StringSubstrColStart(0, 3, 1);
     batch.size = 1;
     expr.evaluate(batch);
     outCol = (BytesColumnVector) batch.cols[1];
@@ -1154,21 +1154,22 @@ public class TestVectorStringExpressions
     v = new BytesColumnVector();
     v.isRepeating = false;
     v.noNulls = true;
-    v.setRef(0, multiByte, 3, 10);
+    
+    // string is 2 chars long (a 3 byte and a 4 byte char)
+    v.setRef(0, multiByte, 3, 7);
     batch.cols[0] = v;
     batch.cols[1] = outV;
     outV.isRepeating = true;
     outV.noNulls = false;
     outCol = (BytesColumnVector) batch.cols[1];
-    expr = new StringSubstrColStart(0, 1, 1);
+    expr = new StringSubstrColStart(0, 2, 1);
     expr.evaluate(batch);
     Assert.assertFalse(outV.isRepeating);
     Assert.assertTrue(outV.noNulls);
     Assert.assertEquals(0,
     StringExpr.compare(
-            // Since references starts at index 3 (2nd char), substring with start idx 1
-            // will start at the 3rd char which starts at index 6
-            multiByte, 6, 10 - 6, outCol.vector[0], outCol.start[0], outCol.length[0]
+            // the result is the last 1 character, which occupies 4 bytes
+            multiByte, 6, 4, outCol.vector[0], outCol.start[0], outCol.length[0]
         )
     );
   }
@@ -1197,7 +1198,7 @@ public class TestVectorStringExpressions
     outV.isRepeating = true;
     outV.noNulls = false;
 
-    StringSubstrColStartLen expr = new StringSubstrColStartLen(0, 5, 6, 1);
+    StringSubstrColStartLen expr = new StringSubstrColStartLen(0, 6, 6, 1);
     expr.evaluate(batch);
     BytesColumnVector outCol = (BytesColumnVector) batch.cols[1];
     Assert.assertEquals(3, batch.size);
@@ -1287,7 +1288,7 @@ public class TestVectorStringExpressions
     outV.isRepeating = true;
     outV.noNulls = false;
 
-    expr = new StringSubstrColStartLen(0, 5, 10, 1);
+    expr = new StringSubstrColStartLen(0, 6, 10, 1);
     expr.evaluate(batch);
     outCol = (BytesColumnVector) batch.cols[1];
     Assert.assertEquals(3, batch.size);
@@ -1368,7 +1369,7 @@ public class TestVectorStringExpressions
     batch.cols[1] = outV;
     outV.isRepeating = true;
     outV.noNulls = false;
-    expr = new StringSubstrColStartLen(0, 2, 2, 1);
+    expr = new StringSubstrColStartLen(0, 3, 2, 1);
     expr.evaluate(batch);
     Assert.assertEquals(1, batch.size);
     Assert.assertFalse(outV.isRepeating);
@@ -1391,7 +1392,7 @@ public class TestVectorStringExpressions
     batch.cols[1] = outV;
     outV.isRepeating = true;
     outV.noNulls = false;
-    expr = new StringSubstrColStartLen(0, 1, 2, 1);
+    expr = new StringSubstrColStartLen(0, 2, 2, 1);
     expr.evaluate(batch);
     outCol = (BytesColumnVector) batch.cols[1];
     Assert.assertEquals(1, batch.size);