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/10/08 17:51:12 UTC

svn commit: r1530318 - in /hive/trunk: ant/src/org/apache/hadoop/hive/ant/ ql/src/gen/vectorization/ExpressionTemplates/ ql/src/java/org/apache/hadoop/hive/ql/exec/vector/ ql/src/test/org/apache/hadoop/hive/ql/exec/vector/ ql/src/test/org/apache/hadoop...

Author: hashutosh
Date: Tue Oct  8 15:51:11 2013
New Revision: 1530318

URL: http://svn.apache.org/r1530318
Log:
HIVE-5365 : Boolean constants in the query are not handled correctly. (Jitendra Nath Pandey via Ashutosh Chauhan)

Added:
    hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
    hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
    hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareColumn.txt
    hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareScalar.txt
    hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareColumn.txt
Modified:
    hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
    hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnCompareColumn.txt
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java

Modified: hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java
URL: http://svn.apache.org/viewvc/hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java?rev=1530318&r1=1530317&r2=1530318&view=diff
==============================================================================
--- hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java (original)
+++ hive/trunk/ant/src/org/apache/hadoop/hive/ant/GenVectorCode.java Tue Oct  8 15:51:11 2013
@@ -117,6 +117,45 @@ public class GenVectorCode extends Task 
       {"ColumnCompareScalar", "GreaterEqual", "long", "double", ">="},
       {"ColumnCompareScalar", "GreaterEqual", "double", "double", ">="},
 
+      {"ColumnCompareScalar", "Equal", "long", "long", "=="},
+      {"ColumnCompareScalar", "Equal", "double", "long", "=="},
+      {"ColumnCompareScalar", "NotEqual", "long", "long", "!="},
+      {"ColumnCompareScalar", "NotEqual", "double", "long", "!="},
+      {"ColumnCompareScalar", "Less", "long", "long", "<"},
+      {"ColumnCompareScalar", "Less", "double", "long", "<"},
+      {"ColumnCompareScalar", "LessEqual", "long", "long", "<="},
+      {"ColumnCompareScalar", "LessEqual", "double", "long", "<="},
+      {"ColumnCompareScalar", "Greater", "long", "long", ">"},
+      {"ColumnCompareScalar", "Greater", "double", "long", ">"},
+      {"ColumnCompareScalar", "GreaterEqual", "long", "long", ">="},
+      {"ColumnCompareScalar", "GreaterEqual", "double", "long", ">="},
+
+      {"ScalarCompareColumn", "Equal", "long", "double", "=="},
+      {"ScalarCompareColumn", "Equal", "double", "double", "=="},
+      {"ScalarCompareColumn", "NotEqual", "long", "double", "!="},
+      {"ScalarCompareColumn", "NotEqual", "double", "double", "!="},
+      {"ScalarCompareColumn", "Less", "long", "double", "<"},
+      {"ScalarCompareColumn", "Less", "double", "double", "<"},
+      {"ScalarCompareColumn", "LessEqual", "long", "double", "<="},
+      {"ScalarCompareColumn", "LessEqual", "double", "double", "<="},
+      {"ScalarCompareColumn", "Greater", "long", "double", ">"},
+      {"ScalarCompareColumn", "Greater", "double", "double", ">"},
+      {"ScalarCompareColumn", "GreaterEqual", "long", "double", ">="},
+      {"ScalarCompareColumn", "GreaterEqual", "double", "double", ">="},
+
+      {"ScalarCompareColumn", "Equal", "long", "long", "=="},
+      {"ScalarCompareColumn", "Equal", "double", "long", "=="},
+      {"ScalarCompareColumn", "NotEqual", "long", "long", "!="},
+      {"ScalarCompareColumn", "NotEqual", "double", "long", "!="},
+      {"ScalarCompareColumn", "Less", "long", "long", "<"},
+      {"ScalarCompareColumn", "Less", "double", "long", "<"},
+      {"ScalarCompareColumn", "LessEqual", "long", "long", "<="},
+      {"ScalarCompareColumn", "LessEqual", "double", "long", "<="},
+      {"ScalarCompareColumn", "Greater", "long", "long", ">"},
+      {"ScalarCompareColumn", "Greater", "double", "long", ">"},
+      {"ScalarCompareColumn", "GreaterEqual", "long", "long", ">="},
+      {"ScalarCompareColumn", "GreaterEqual", "double", "long", ">="},
+
       {"FilterColumnCompareScalar", "Equal", "long", "double", "=="},
       {"FilterColumnCompareScalar", "Equal", "double", "double", "=="},
       {"FilterColumnCompareScalar", "NotEqual", "long", "double", "!="},
@@ -176,6 +215,13 @@ public class GenVectorCode extends Task 
       {"FilterStringColumnCompareScalar", "Greater", ">"},
       {"FilterStringColumnCompareScalar", "GreaterEqual", ">="},
 
+      {"StringColumnCompareScalar", "Equal", "=="},
+      {"StringColumnCompareScalar", "NotEqual", "!="},
+      {"StringColumnCompareScalar", "Less", "<"},
+      {"StringColumnCompareScalar", "LessEqual", "<="},
+      {"StringColumnCompareScalar", "Greater", ">"},
+      {"StringColumnCompareScalar", "GreaterEqual", ">="},
+
       {"FilterStringScalarCompareColumn", "Equal", "=="},
       {"FilterStringScalarCompareColumn", "NotEqual", "!="},
       {"FilterStringScalarCompareColumn", "Less", "<"},
@@ -183,6 +229,13 @@ public class GenVectorCode extends Task 
       {"FilterStringScalarCompareColumn", "Greater", ">"},
       {"FilterStringScalarCompareColumn", "GreaterEqual", ">="},
 
+      {"StringScalarCompareColumn", "Equal", "=="},
+      {"StringScalarCompareColumn", "NotEqual", "!="},
+      {"StringScalarCompareColumn", "Less", "<"},
+      {"StringScalarCompareColumn", "LessEqual", "<="},
+      {"StringScalarCompareColumn", "Greater", ">"},
+      {"StringScalarCompareColumn", "GreaterEqual", ">="},
+
       {"FilterStringColumnCompareColumn", "Equal", "=="},
       {"FilterStringColumnCompareColumn", "NotEqual", "!="},
       {"FilterStringColumnCompareColumn", "Less", "<"},
@@ -190,6 +243,13 @@ public class GenVectorCode extends Task 
       {"FilterStringColumnCompareColumn", "Greater", ">"},
       {"FilterStringColumnCompareColumn", "GreaterEqual", ">="},
 
+      {"StringColumnCompareColumn", "Equal", "=="},
+      {"StringColumnCompareColumn", "NotEqual", "!="},
+      {"StringColumnCompareColumn", "Less", "<"},
+      {"StringColumnCompareColumn", "LessEqual", "<="},
+      {"StringColumnCompareColumn", "Greater", ">"},
+      {"StringColumnCompareColumn", "GreaterEqual", ">="},
+
       {"FilterColumnCompareColumn", "Equal", "long", "double", "=="},
       {"FilterColumnCompareColumn", "Equal", "double", "double", "=="},
       {"FilterColumnCompareColumn", "NotEqual", "long", "double", "!="},
@@ -216,6 +276,32 @@ public class GenVectorCode extends Task 
         {"FilterColumnCompareColumn", "GreaterEqual", "long", "long", ">="},
         {"FilterColumnCompareColumn", "GreaterEqual", "double", "long", ">="},
 
+      {"ColumnCompareColumn", "Equal", "long", "double", "=="},
+      {"ColumnCompareColumn", "Equal", "double", "double", "=="},
+      {"ColumnCompareColumn", "NotEqual", "long", "double", "!="},
+      {"ColumnCompareColumn", "NotEqual", "double", "double", "!="},
+      {"ColumnCompareColumn", "Less", "long", "double", "<"},
+      {"ColumnCompareColumn", "Less", "double", "double", "<"},
+      {"ColumnCompareColumn", "LessEqual", "long", "double", "<="},
+      {"ColumnCompareColumn", "LessEqual", "double", "double", "<="},
+      {"ColumnCompareColumn", "Greater", "long", "double", ">"},
+      {"ColumnCompareColumn", "Greater", "double", "double", ">"},
+      {"ColumnCompareColumn", "GreaterEqual", "long", "double", ">="},
+      {"ColumnCompareColumn", "GreaterEqual", "double", "double", ">="},
+
+      {"ColumnCompareColumn", "Equal", "long", "long", "=="},
+      {"ColumnCompareColumn", "Equal", "double", "long", "=="},
+      {"ColumnCompareColumn", "NotEqual", "long", "long", "!="},
+      {"ColumnCompareColumn", "NotEqual", "double", "long", "!="},
+      {"ColumnCompareColumn", "Less", "long", "long", "<"},
+      {"ColumnCompareColumn", "Less", "double", "long", "<"},
+      {"ColumnCompareColumn", "LessEqual", "long", "long", "<="},
+      {"ColumnCompareColumn", "LessEqual", "double", "long", "<="},
+      {"ColumnCompareColumn", "Greater", "long", "long", ">"},
+      {"ColumnCompareColumn", "Greater", "double", "long", ">"},
+      {"ColumnCompareColumn", "GreaterEqual", "long", "long", ">="},
+      {"ColumnCompareColumn", "GreaterEqual", "double", "long", ">="},
+
       // template, <ClassNamePrefix>, <ReturnType>, <OperandType>, <FuncName>, <OperandCast>,
       //   <ResultCast>
       {"ColumnUnaryFunc", "FuncRound", "double", "double", "MathExpr.round", "", ""},
@@ -380,6 +466,8 @@ public class GenVectorCode extends Task 
         generateColumnArithmeticScalar(tdesc);
       } else if (tdesc[0].equals("ColumnCompareScalar")) {
         generateColumnCompareScalar(tdesc);
+      } else if (tdesc[0].equals("ScalarCompareColumn")) {
+        generateScalarCompareColumn(tdesc);
       } else if (tdesc[0].equals("FilterColumnCompareScalar")) {
         generateFilterColumnCompareScalar(tdesc);
       } else if (tdesc[0].equals("FilterScalarCompareColumn")) {
@@ -388,6 +476,8 @@ public class GenVectorCode extends Task 
         generateScalarArithmeticColumn(tdesc);
       } else if (tdesc[0].equals("FilterColumnCompareColumn")) {
         generateFilterColumnCompareColumn(tdesc);
+      } else if (tdesc[0].equals("ColumnCompareColumn")) {
+        generateColumnCompareColumn(tdesc);
       } else if (tdesc[0].equals("ColumnArithmeticColumn")) {
         generateColumnArithmeticColumn(tdesc);
       } else if (tdesc[0].equals("ColumnUnaryMinus")) {
@@ -406,10 +496,16 @@ public class GenVectorCode extends Task 
         generateVectorUDAFVar(tdesc);
       } else if (tdesc[0].equals("FilterStringColumnCompareScalar")) {
         generateFilterStringColumnCompareScalar(tdesc);
+      } else if (tdesc[0].equals("StringColumnCompareScalar")) {
+        generateStringColumnCompareScalar(tdesc);
       } else if (tdesc[0].equals("FilterStringScalarCompareColumn")) {
         generateFilterStringScalarCompareColumn(tdesc);
+      } else if (tdesc[0].equals("StringScalarCompareColumn")) {
+        generateStringScalarCompareColumn(tdesc);
       } else if (tdesc[0].equals("FilterStringColumnCompareColumn")) {
         generateFilterStringColumnCompareColumn(tdesc);
+      } else if (tdesc[0].equals("StringColumnCompareColumn")) {
+        generateStringColumnCompareColumn(tdesc);
       } else {
         continue;
       }
@@ -418,6 +514,17 @@ public class GenVectorCode extends Task 
     testCodeGen.generateTestSuites();
   }
 
+  private void generateColumnCompareColumn(String[] tdesc) throws IOException {
+    //The variables are all same as ColumnCompareScalar except that
+    //this template doesn't need a return type. Pass anything as return type.
+    String operatorName = tdesc[1];
+    String operandType1 = tdesc[2];
+    String operandType2 = tdesc[3];
+    String className = getCamelCaseType(operandType1)
+        + "Col" + operatorName + getCamelCaseType(operandType2) + "Column";
+    generateColumnBinaryOperatorColumn(tdesc, "long", className);
+  }
+
   private void generateVectorUDAFMinMax(String[] tdesc) throws Exception {
     String className = tdesc[1];
     String valueType = tdesc[2];
@@ -523,22 +630,42 @@ public class GenVectorCode extends Task 
     String className = "FilterStringScalar" + operatorName + "StringColumn";
 
     // Template expansion logic is the same for both column-scalar and scalar-column cases.
-    generateFilterStringColumnCompareScalar(tdesc, className);
+    generateStringColumnCompareScalar(tdesc, className);
+  }
+
+  private void generateStringScalarCompareColumn(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String className = "StringScalar" + operatorName + "StringColumn";
+
+    // Template expansion logic is the same for both column-scalar and scalar-column cases.
+    generateStringColumnCompareScalar(tdesc, className);
   }
 
   private void generateFilterStringColumnCompareScalar(String[] tdesc) throws IOException {
     String operatorName = tdesc[1];
     String className = "FilterStringCol" + operatorName + "StringScalar";
-    generateFilterStringColumnCompareScalar(tdesc, className);
+    generateStringColumnCompareScalar(tdesc, className);
+  }
+
+  private void generateStringColumnCompareScalar(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String className = "StringCol" + operatorName + "StringScalar";
+    generateStringColumnCompareScalar(tdesc, className);
   }
 
   private void generateFilterStringColumnCompareColumn(String[] tdesc) throws IOException {
     String operatorName = tdesc[1];
     String className = "FilterStringCol" + operatorName + "StringColumn";
-    generateFilterStringColumnCompareScalar(tdesc, className);
+    generateStringColumnCompareScalar(tdesc, className);
   }
 
-  private void generateFilterStringColumnCompareScalar(String[] tdesc, String className)
+  private void generateStringColumnCompareColumn(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String className = "StringCol" + operatorName + "StringColumn";
+    generateStringColumnCompareScalar(tdesc, className);
+  }
+
+  private void generateStringColumnCompareScalar(String[] tdesc, String className)
       throws IOException {
    String operatorSymbol = tdesc[2];
    String outputFile = joinPath(this.expressionOutputDirectory, className + ".java");
@@ -648,6 +775,16 @@ public class GenVectorCode extends Task 
     generateColumnBinaryOperatorScalar(tdesc, returnType, className);
   }
 
+  private void generateScalarCompareColumn(String[] tdesc) throws IOException {
+    String operatorName = tdesc[1];
+    String operandType1 = tdesc[2];
+    String operandType2 = tdesc[3];
+    String returnType = "long";
+    String className = getCamelCaseType(operandType1)
+        + "Scalar" + operatorName + getCamelCaseType(operandType2) + "Column";
+    generateScalarBinaryOperatorColumn(tdesc, returnType, className);
+  }
+
   private void generateColumnBinaryOperatorColumn(String[] tdesc, String returnType,
          String className) throws IOException {
     String operandType1 = tdesc[2];

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt?rev=1530318&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ColumnCompareColumn.txt Tue Oct  8 15:51:11 2013
@@ -0,0 +1,157 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.NullUtil;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Generated from template ColumnArithmeticColumn.txt, which covers binary arithmetic 
+ * expressions between columns.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+  
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    <InputColumnVectorType1> inputColVector1 = (<InputColumnVectorType1>) batch.cols[colNum1];
+    <InputColumnVectorType2> inputColVector2 = (<InputColumnVectorType2>) batch.cols[colNum2];
+    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    <OperandType1>[] vector1 = inputColVector1.vector;
+    <OperandType2>[] vector2 = inputColVector2.vector;
+    <ReturnType>[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    outputColVector.isRepeating = 
+         inputColVector1.isRepeating && inputColVector2.isRepeating
+      || inputColVector1.isRepeating && !inputColVector1.noNulls && inputColVector1.isNull[0]
+      || inputColVector2.isRepeating && !inputColVector2.noNulls && inputColVector2.isNull[0];
+    
+    // Handle nulls first  
+    NullUtil.propagateNullsColCol(
+      inputColVector1, inputColVector2, outputColVector, sel, n, batch.selectedInUse);
+          
+    /* Disregard nulls for processing. In other words,
+     * the arithmetic operation is performed even if one or 
+     * more inputs are null. This is to improve speed by avoiding
+     * conditional checks in the inner loop.
+     */ 
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      outputVector[0] = vector1[0] <OperatorSymbol> vector2[0] ? 1 : 0;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] <OperatorSymbol> vector2[0] ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] <OperatorSymbol> vector2[0] ? 1 : 0;
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] <OperatorSymbol> vector2[i] ? 1 : 0;
+        }
+      }
+    }
+    
+    /* For the case when the output can have null values, follow 
+     * the convention that the data values must be 1 for long and 
+     * NaN for double. This is to prevent possible later zero-divide errors
+     * in complex arithmetic expressions like col2 / (col1 - 1)
+     * in the case when some col1 entries are null.
+     */
+    NullUtil.setNullDataEntries<CamelReturnType>(outputColVector, batch.selectedInUse, sel, n);
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "<ReturnType>";
+  }
+  
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+}

Modified: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnCompareColumn.txt?rev=1530318&r1=1530317&r2=1530318&view=diff
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnCompareColumn.txt (original)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/FilterStringColumnCompareColumn.txt Tue Oct  8 15:51:11 2013
@@ -303,7 +303,7 @@ public class <ClassName> extends VectorE
         } else {
           int newSize = 0;
           for(int i = 0; i != n; i++) {
-            if (!nullPos2[i]) {
+            if (!nullPos1[i]) {
               if (StringExpr.compare(vector1[i], start1[i], length1[i], 
                                      vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
                 sel[newSize++] = i;

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt?rev=1530318&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/ScalarCompareColumn.txt Tue Oct  8 15:51:11 2013
@@ -0,0 +1,149 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.<InputColumnVectorType>;
+import org.apache.hadoop.hive.ql.exec.vector.<OutputColumnVectorType>;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Generated from template ColumnCompareScalar.txt, which covers binary comparison 
+ * expressions between a column and a scalar. The boolean output is stored in a 
+ * separate boolean column.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private <OperandType1> value;
+  private int outputColumn;
+
+  public <ClassName>(<OperandType1> value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    <InputColumnVectorType> inputColVector = (<InputColumnVectorType>) batch.cols[colNum];
+    <OutputColumnVectorType> outputColVector = (<OutputColumnVectorType>) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNulls = outputColVector.isNull;
+    int n = batch.size;
+    <OperandType2>[] vector = inputColVector.vector;
+    <ReturnType>[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    outputColVector.isRepeating = false;
+    outputColVector.noNulls = inputColVector.noNulls;
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = value <OperatorSymbol> vector[0] ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value <OperatorSymbol> vector[i] ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value <OperatorSymbol> vector[i] ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = value <OperatorSymbol> vector[0] ? 1 : 0;
+          outNulls[0] = false;
+        } else {
+          outNulls[0] = true;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = value <OperatorSymbol> vector[i] ? 1 : 0;
+            outNulls[i] = false;
+          } else {
+            //comparison with null is null
+            outNulls[i] = true;
+          }
+        }
+      } else {
+        System.arraycopy(nullPos, 0, outNulls, 0, n);
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = value <OperatorSymbol> vector[i] ? 1 : 0;
+          } 
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "<ReturnType>";
+  }
+  
+  public int getColNum() {
+    return colNum;
+  }
+  
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public <OperandType1> getValue() {
+    return value;
+  }
+
+  public void setValue(<OperandType1> value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+}

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareColumn.txt?rev=1530318&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareColumn.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareColumn.txt Tue Oct  8 15:51:11 2013
@@ -0,0 +1,493 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Filter the rows in a batch by comparing one string column to another. 
+ * This code is generated from a template.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum1;
+  private int colNum2;
+  private int outputColumn;
+
+  public <ClassName>(int colNum1, int colNum2, int outputColumn) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    BytesColumnVector inputColVector1 = (BytesColumnVector) batch.cols[colNum1];
+    BytesColumnVector inputColVector2 = (BytesColumnVector) batch.cols[colNum2];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    boolean[] outNull = outputColVector.isNull;
+
+    int n = batch.size;
+    byte[][] vector1 = inputColVector1.vector;
+    byte[][] vector2 = inputColVector2.vector;
+    int[] start1 = inputColVector1.start;
+    int[] start2 = inputColVector2.start;
+    int[] length1 = inputColVector1.length;
+    int[] length2 = inputColVector2.length;
+
+    long[] outVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+   
+    outputColVector.noNulls = true;
+    outputColVector.isRepeating = false;
+    // handle case where neither input has nulls
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = true;
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        outputColVector.isRepeating = true;
+        int ret = StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                         vector2[0], start2[0], length2[0]);
+        if (ret <OperatorSymbol> 0) {
+          outVector[0] = 1;
+        } else {
+          outVector[0] = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      } else if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                 vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      }
+    
+    // handle case where only input 2 has nulls
+    } else if (inputColVector1.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        outputColVector.isRepeating = true;
+        outNull[0] = nullPos2[0];
+        if (!nullPos2[0]) {
+          if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+            outVector[0] = 1;
+          } else {
+            outVector[0] = 0;
+          }
+        } 
+      } else if (inputColVector1.isRepeating) {
+          
+         // no need to check for nulls in input 1
+         if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = nullPos2[i];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            } 
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = nullPos2[i];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            } 
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          // Entire output vector will be null
+          outputColVector.isRepeating = true;
+          outNull[0] = true;
+          return;
+        }
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = false;
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = false;
+            if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                   vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      } else { // neither input is repeating
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = nullPos2[i];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            } 
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = nullPos2[i];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        }      
+      }
+      
+    // handle case where only input 1 has nulls
+    } else if (inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        outputColVector.isRepeating = true;
+        outNull[0] = nullPos1[0];
+        if (!nullPos1[0]) {
+          if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                               vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+            outVector[0] = 1;
+          } else {
+            outVector[0] = 0;
+          }
+        } 
+      } else if (inputColVector1.isRepeating) {
+        if (nullPos1[0]) {
+          // Entire output vector will be null
+          outputColVector.isRepeating = true;
+          outNull[0] = true;
+          return; 
+        }
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = false;
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = false;
+            if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                   vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+         if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = nullPos1[i];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = nullPos1[i];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = nullPos1[i];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = nullPos1[i];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        }      
+      }
+            
+    // handle case where both inputs have nulls
+    } else {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        outputColVector.isRepeating = true;
+        outNull[0] = nullPos1[0] || nullPos2[0];
+        if (!outNull[0]) {
+          if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                             vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+            outVector[0] = 1;
+          } else {
+            outVector[0] = 0;
+          }
+        } 
+      } else if (inputColVector1.isRepeating) {
+         if (nullPos1[0]) {
+           outputColVector.isRepeating = true;
+           outNull[0] = true;
+           return;
+         }
+         if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = nullPos2[i];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = nullPos2[i];
+            if (!nullPos2[i]) {
+              if (StringExpr.compare(vector1[0], start1[0], length1[0], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (nullPos2[0]) {
+          outputColVector.isRepeating = true;
+          outNull[0] = true;
+          return;
+        }
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = nullPos1[i];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = nullPos1[i];
+            if (!nullPos1[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[0], start2[0], length2[0]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        }
+      } else { // neither input is repeating
+         if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outNull[i] = nullPos1[i] || nullPos2[i];
+            if (!outNull[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outNull[i] = nullPos1[i] || nullPos2[i];
+            if (!outNull[i]) {
+              if (StringExpr.compare(vector1[i], start1[i], length1[i], 
+                                     vector2[i], start2[i], length2[i]) <OperatorSymbol> 0) {
+                outVector[i] = 1;
+              } else {
+                outVector[i] = 0;
+              }
+            }
+          }
+        }      
+      } 
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+  
+  public int getColNum1() {
+    return colNum1;
+  }
+
+  public void setColNum1(int colNum1) {
+    this.colNum1 = colNum1;
+  }
+
+  public int getColNum2() {
+    return colNum2;
+  }
+
+  public void setColNum2(int colNum2) {
+    this.colNum2 = colNum2;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+}

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareScalar.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareScalar.txt?rev=1530318&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareScalar.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringColumnCompareScalar.txt Tue Oct  8 15:51:11 2013
@@ -0,0 +1,165 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private byte[] value;
+  private int outputColumn;
+
+  public <ClassName>(int colNum, byte[] value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNull = outputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    long[] outVector = outputColVector.vector;
+    
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+   
+    outputColVector.isRepeating = false; 
+    if (inputColVector.noNulls) {
+      outputColVector.noNulls = true;
+      if (inputColVector.isRepeating) {
+        outputColVector.isRepeating = true; 
+        if (StringExpr.compare(vector[0], start[0], length[0], value, 0, value.length) <OperatorSymbol> 0) {
+          outVector[0] = 1;
+        } else {
+          outVector[0] = 0;
+        }
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(vector[i], start[i], length[i], value, 0, value.length) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(vector[i], start[i], length[i], value, 0, value.length) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      }
+    } else {
+      outputColVector.noNulls = false;
+      if (inputColVector.isRepeating) {
+        outputColVector.isRepeating = true;
+        outNull[0] = nullPos[0];
+        if (!nullPos[0]) {
+          if (StringExpr.compare(vector[0], start[0], length[0], value, 0, value.length) <OperatorSymbol> 0) {
+            outVector[0] = 1;
+          } else {
+            outVector[0] = 0;
+          }
+        } 
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outNull[i] = nullPos[i];
+          if (!nullPos[i]) {
+            if (StringExpr.compare(vector[i], start[i], length[i], value, 0, value.length) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outNull[i] = nullPos[i];
+          if (!nullPos[i]) {
+            if (StringExpr.compare(vector[i], start[i], length[i], value, 0, value.length) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+  
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public byte[] getValue() {
+    return value;
+  }
+
+  public void setValue(byte[] value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+}

Added: hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareColumn.txt
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareColumn.txt?rev=1530318&view=auto
==============================================================================
--- hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareColumn.txt (added)
+++ hive/trunk/ql/src/gen/vectorization/ExpressionTemplates/StringScalarCompareColumn.txt Tue Oct  8 15:51:11 2013
@@ -0,0 +1,165 @@
+/**
+ * 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.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * This is a generated class to evaluate a <OperatorSymbol> comparison on a vector of strings.
+ * Do not edit the generated code directly. 
+ */
+public class <ClassName> extends VectorExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private int colNum;
+  private byte[] value;
+  private int outputColumn;
+
+  public <ClassName>(int colNum, byte[] value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  public <ClassName>() {
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+    BytesColumnVector inputColVector = (BytesColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    boolean[] outNull = outputColVector.isNull;
+    int n = batch.size;
+    byte[][] vector = inputColVector.vector;
+    int[] length = inputColVector.length;
+    int[] start = inputColVector.start;
+    long[] outVector = outputColVector.vector; 
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+   
+    outputColVector.isRepeating = false;
+    if (inputColVector.noNulls) {
+      outputColVector.noNulls = true;
+      if (inputColVector.isRepeating) {
+        outputColVector.isRepeating = true; 
+        if (StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <OperatorSymbol> 0) {
+          outVector[0] = 1;
+        } else {
+          outVector[0] = 0;
+        }
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+            outVector[i] = 1;
+          } else {
+            outVector[i] = 0;
+          }
+        }
+      }
+    } else {
+      outputColVector.noNulls = false;
+      if (inputColVector.isRepeating) {
+        outputColVector.isRepeating = true;
+        outNull[0] = nullPos[0];
+        if (!nullPos[0]) {
+          if (StringExpr.compare(value, 0, value.length, vector[0], start[0], length[0]) <OperatorSymbol> 0) {
+            outVector[0] = 1;
+          } else {
+            outVector[0] = 0;
+          }
+        } 
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outNull[i] = nullPos[i];
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outNull[i] = nullPos[i];
+          if (!nullPos[i]) {
+            if (StringExpr.compare(value, 0, value.length, vector[i], start[i], length[i]) <OperatorSymbol> 0) {
+              outVector[i] = 1;
+            } else {
+              outVector[i] = 0;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+  
+  public int getColNum() {
+    return colNum;
+  }
+
+  public void setColNum(int colNum) {
+    this.colNum = colNum;
+  }
+
+  public byte[] getValue() {
+    return value;
+  }
+
+  public void setValue(byte[] value) {
+    this.value = value;
+  }
+
+  public void setOutputColumn(int outputColumn) {
+    this.outputColumn = outputColumn;
+  }
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java?rev=1530318&r1=1530317&r2=1530318&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java Tue Oct  8 15:51:11 2013
@@ -81,7 +81,6 @@ import org.apache.hadoop.hive.ql.plan.Ex
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
-import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
 import org.apache.hadoop.hive.ql.udf.UDFDayOfMonth;
 import org.apache.hadoop.hive.ql.udf.UDFHour;
 import org.apache.hadoop.hive.ql.udf.UDFLTrim;
@@ -105,6 +104,7 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.UDFYear;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFConcat;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFLower;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
@@ -149,25 +149,9 @@ public class VectorizationContext {
   }
 
   private int getInputColumnIndex(String name) {
-    if (columnMap == null) {
-      //Null is treated as test call, is used for validation test.
-      return 0;
-    } else {
       return columnMap.get(name);
-    }
   }
 
-  /* Return true if we are running in the planner, and false if we
-   * are running in a task.
-   */
-  /*
-  private boolean isPlanner() {
-
-    // This relies on the behavior that columnMap is null in the planner.
-    return columnMap == null;
-  }
-  */
-
   private class OutputColumnManager {
     private final int initialOutputCol;
     private int outputColCount = 0;
@@ -1161,6 +1145,12 @@ public class VectorizationContext {
       } catch (Exception ex) {
         throw new HiveException(ex);
       }
+    } else if (constDesc.getTypeString().equalsIgnoreCase("boolean")) {
+      if (constDesc.getValue().equals(Boolean.valueOf(true))) {
+        return 1;
+      } else {
+        return 0;
+      }
     } else {
       return constDesc.getValue();
     }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java?rev=1530318&r1=1530317&r2=1530318&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java Tue Oct  8 15:51:11 2013
@@ -31,6 +31,7 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.DoubleColUnaryMinus;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterDoubleColLessDoubleScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterLongColEqualLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterLongColGreaterLongScalar;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterLongScalarGreaterLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringColGreaterStringColumn;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hive.ql.udf.UDF
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPAnd;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPEqual;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPGreaterThan;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPLessThan;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPOr;
@@ -398,4 +400,29 @@ public class TestVectorizationContext {
     VectorExpression ve = vc.getVectorExpression(scalarGreaterColExpr);
     assertEquals(FilterLongScalarGreaterLongColumn.class, ve.getClass());
   }
+
+  @Test
+  public void testFilterBooleanColumnCompareBooleanScalar() throws HiveException {
+    ExprNodeGenericFuncDesc colEqualScalar = new ExprNodeGenericFuncDesc();
+    GenericUDFOPEqual gudf = new GenericUDFOPEqual();
+    colEqualScalar.setGenericUDF(gudf);
+    List<ExprNodeDesc> children = new ArrayList<ExprNodeDesc>(2);
+    ExprNodeConstantDesc constDesc =
+        new ExprNodeConstantDesc(TypeInfoFactory.booleanTypeInfo, 20);
+    ExprNodeColumnDesc colDesc =
+        new ExprNodeColumnDesc(Boolean.class, "a", "table", false);
+
+    children.add(colDesc);
+    children.add(constDesc);
+
+    colEqualScalar.setChildExprs(children);
+
+    Map<String, Integer> columnMap = new HashMap<String, Integer>();
+    columnMap.put("a", 0);
+
+    VectorizationContext vc = new VectorizationContext(columnMap, 2);
+    vc.setOperatorType(OperatorType.FILTER);
+    VectorExpression ve = vc.getVectorExpression(colEqualScalar);
+    assertEquals(FilterLongColEqualLongScalar.class, ve.getClass());
+  }
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java?rev=1530318&r1=1530317&r2=1530318&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java Tue Oct  8 15:51:11 2013
@@ -340,27 +340,6 @@ public class TestVectorLogicalExpression
   }
 
   @Test
-  public void testFilterNotExpr() {
-    VectorizedRowBatch batch1 = getBatchThreeBooleanCols();
-    VectorizedRowBatch batch2 = getBatchThreeBooleanCols();
-
-    SelectColumnIsTrue expr = new SelectColumnIsTrue(0);
-    FilterNotExpr notExpr = new FilterNotExpr(expr);
-
-    notExpr.evaluate(batch1);
-
-    notExpr.evaluate(batch2);
-
-    assertEquals(batch1.size, batch2.size);
-    for (int j = 0; j < batch1.size; j++) {
-      assertEquals(batch1.selected[j], batch2.selected[j]);
-      int i = j;
-      assertEquals((((LongColumnVector) batch1.cols[0]).vector[i]),
-          (((LongColumnVector) batch2.cols[0]).vector[i]));
-    }
-  }
-
-  @Test
   public void testFilterExprOrExpr() {
     VectorizedRowBatch batch1 = getBatchThreeBooleanCols();
     VectorizedRowBatch batch2 = getBatchThreeBooleanCols();

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java?rev=1530318&r1=1530317&r2=1530318&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorStringExpressions.java Tue Oct  8 15:51:11 2013
@@ -33,6 +33,9 @@ import org.apache.hadoop.hive.ql.exec.ve
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringScalarEqualStringColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringScalarGreaterStringColumn;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FilterStringScalarLessEqualStringColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.StringColEqualStringScalar;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.StringColLessStringColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.StringScalarEqualStringColumn;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
@@ -179,6 +182,29 @@ public class TestVectorStringExpressions
   }
 
   @Test
+  public void testStringColCompareStringScalarProjection() {
+    VectorizedRowBatch batch = makeStringBatch();
+    VectorExpression expr;
+
+    expr = new StringColEqualStringScalar(0, red2, 2);
+    expr.evaluate(batch);
+    Assert.assertEquals(3, batch.size);
+    LongColumnVector outVector = (LongColumnVector) batch.cols[2];
+    Assert.assertEquals(1, outVector.vector[0]);
+    Assert.assertEquals(0, outVector.vector[1]);
+    Assert.assertEquals(0, outVector.vector[2]);
+
+    batch = makeStringBatch();
+    expr = new StringColEqualStringScalar(0, green, 2);
+    expr.evaluate(batch);
+    Assert.assertEquals(3, batch.size);
+    outVector = (LongColumnVector) batch.cols[2];
+    Assert.assertEquals(0, outVector.vector[0]);
+    Assert.assertEquals(1, outVector.vector[1]);
+    Assert.assertEquals(0, outVector.vector[2]);
+  }
+
+  @Test
   // Test string literal to string column comparison
   public void testStringScalarCompareStringCol() {
     VectorizedRowBatch batch = makeStringBatch();
@@ -207,7 +233,30 @@ public class TestVectorStringExpressions
     Assert.assertTrue(batch.selected[0] == 0);
     Assert.assertTrue(batch.selected[1] == 1);
   }
-  
+
+  @Test
+  public void testStringScalarCompareStringColProjection() {
+    VectorizedRowBatch batch = makeStringBatch();
+    VectorExpression expr;
+
+    expr = new StringScalarEqualStringColumn(0, red2, 2);
+    expr.evaluate(batch);
+    Assert.assertEquals(3, batch.size);
+    LongColumnVector outVector = (LongColumnVector) batch.cols[2];
+    Assert.assertEquals(1, outVector.vector[0]);
+    Assert.assertEquals(0, outVector.vector[1]);
+    Assert.assertEquals(0, outVector.vector[2]);
+
+    batch = makeStringBatch();
+    expr = new StringScalarEqualStringColumn(0, green, 2);
+    expr.evaluate(batch);
+    Assert.assertEquals(3, batch.size);
+    outVector = (LongColumnVector) batch.cols[2];
+    Assert.assertEquals(0, outVector.vector[0]);
+    Assert.assertEquals(1, outVector.vector[1]);
+    Assert.assertEquals(0, outVector.vector[2]);
+  }
+
   @Test
   public void testStringColCompareStringColFilter() {
     VectorizedRowBatch batch;
@@ -301,7 +350,8 @@ public class TestVectorStringExpressions
     batch.cols[1].isRepeating = true;
     batch.cols[1].noNulls = true;
     expr.evaluate(batch);
-    Assert.assertEquals(3, batch.size);
+    Assert.assertEquals(2, batch.size);
+    Assert.assertEquals(0, batch.selected[0]);
     Assert.assertEquals(1, batch.selected[1]);
 
     // left and right repeat
@@ -359,12 +409,269 @@ public class TestVectorStringExpressions
     Assert.assertEquals(0, batch.size);
   }
 
+  @Test
+  public void testStringColCompareStringColProjection() {
+    VectorizedRowBatch batch;
+    VectorExpression expr;
+    long [] outVector;
+
+    /* input data
+     *
+     * col0       col1
+     * ===============
+     * blue       red
+     * green      green
+     * red        blue
+     * NULL       red            col0 data is empty string if we un-set NULL property
+     */
+
+    // nulls possible on left, right
+    batch = makeStringBatchForColColCompare();
+    expr = new StringColLessStringColumn(0, 1, 3);
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(0, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertTrue(batch.cols[3].isNull[3]);
+
+    // no nulls possible
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].noNulls = true;
+    batch.cols[1].noNulls = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertTrue(batch.cols[3].noNulls);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(0, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertFalse(batch.cols[3].isNull[3]);
+    Assert.assertEquals(1, outVector[3]);
+
+    // nulls on left, no nulls on right
+    batch = makeStringBatchForColColCompare();
+    batch.cols[1].noNulls = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(0, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertTrue(batch.cols[3].isNull[3]);
+
+    // nulls on right, no nulls on left
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].noNulls = true;
+    batch.cols[1].isNull[3] = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(0, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertTrue(batch.cols[3].isNull[3]);
+
+    // Now vary isRepeating
+    // nulls possible on left, right
+
+    // left repeats
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(1, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertFalse(batch.cols[3].isNull[3]);
+    Assert.assertEquals(1, outVector[3]);
+
+
+    // right repeats
+    batch = makeStringBatchForColColCompare();
+    batch.cols[1].isRepeating = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(1, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertTrue(batch.cols[3].isNull[3]);
+
+    // left and right repeat
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    batch.cols[1].isRepeating = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertTrue(batch.cols[3].isRepeating);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+
+    // Now vary isRepeating
+    // nulls possible only on left
+
+    // left repeats
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    batch.cols[1].noNulls = true;
+    expr.evaluate(batch);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertEquals(4, batch.size);
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(1, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertFalse(batch.cols[3].isNull[3]);
+    Assert.assertEquals(1, outVector[3]);
+
+    // left repeats and is null
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    batch.cols[1].noNulls = true;
+    batch.cols[0].isNull[0] = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertTrue(batch.cols[3].isRepeating);
+    Assert.assertTrue(batch.cols[3].isNull[0]);
+
+    // right repeats
+    batch = makeStringBatchForColColCompare();
+    batch.cols[1].isRepeating = true;
+    batch.cols[1].noNulls = true;
+    expr.evaluate(batch);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertEquals(4, batch.size);
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(1, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertTrue(batch.cols[3].isNull[3]);
+
+    // left and right repeat
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    batch.cols[1].isRepeating = true;
+    batch.cols[1].noNulls = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertTrue(batch.cols[3].isRepeating);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+
+
+
+    // Now vary isRepeating
+    // nulls possible only on right
+
+    // left repeats
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    batch.cols[0].noNulls = true;
+    batch.cols[1].isNull[0] = true;
+    expr.evaluate(batch);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertEquals(4, batch.size);
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertTrue(batch.cols[3].isNull[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(1, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertFalse(batch.cols[3].isNull[3]);
+    Assert.assertEquals(1, outVector[3]);
+
+    // right repeats
+    batch = makeStringBatchForColColCompare();
+    batch.cols[1].isRepeating = true;
+    batch.cols[0].noNulls = true;
+    expr.evaluate(batch);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertEquals(4, batch.size);
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+    Assert.assertFalse(batch.cols[3].isNull[1]);
+    Assert.assertEquals(1, outVector[1]);
+    Assert.assertFalse(batch.cols[3].isNull[2]);
+    Assert.assertEquals(0, outVector[2]);
+    Assert.assertFalse(batch.cols[3].isNull[3]);
+    Assert.assertEquals(1, outVector[3]);
+
+    // right repeats and is null
+    batch = makeStringBatchForColColCompare();
+    batch.cols[1].isRepeating = true;
+    batch.cols[0].noNulls = true;
+    batch.cols[1].isNull[0] = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertTrue(batch.cols[3].isRepeating);
+    Assert.assertTrue(batch.cols[3].isNull[0]);
+
+    // left and right repeat
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    batch.cols[1].isRepeating = true;
+    batch.cols[0].noNulls = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    outVector = ((LongColumnVector) batch.cols[3]).vector;
+    Assert.assertTrue(batch.cols[3].isRepeating);
+    Assert.assertFalse(batch.cols[3].isNull[0]);
+    Assert.assertEquals(1, outVector[0]);
+
+    // left and right repeat and right is null
+    batch = makeStringBatchForColColCompare();
+    batch.cols[0].isRepeating = true;
+    batch.cols[1].isRepeating = true;
+    batch.cols[0].noNulls = true;
+    batch.cols[1].isNull[0] = true;
+    expr.evaluate(batch);
+    Assert.assertEquals(4, batch.size);
+    Assert.assertFalse(batch.cols[3].noNulls);
+    Assert.assertTrue(batch.cols[3].isRepeating);
+    Assert.assertTrue(batch.cols[3].isNull[0]);
+  }
+
   VectorizedRowBatch makeStringBatch() {
     // create a batch with one string ("Bytes") column
-    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    VectorizedRowBatch batch = new VectorizedRowBatch(3);
     BytesColumnVector v = new BytesColumnVector();
     batch.cols[0] = v;
     batch.cols[1] = new BytesColumnVector();          // to hold output if needed
+    batch.cols[2] = new LongColumnVector(batch.size); // to hold boolean output
     /*
      * Add these 3 values:
      *
@@ -580,12 +887,13 @@ public class TestVectorStringExpressions
   }
 
   private VectorizedRowBatch makeStringBatchForColColCompare() {
-    VectorizedRowBatch batch = new VectorizedRowBatch(3);
+    VectorizedRowBatch batch = new VectorizedRowBatch(4);
     BytesColumnVector v = new BytesColumnVector();
     batch.cols[0] = v;
     BytesColumnVector v2 = new BytesColumnVector();
     batch.cols[1] = v2;
     batch.cols[2] = new BytesColumnVector();
+    batch.cols[3] = new LongColumnVector();
 
     v.setRef(0, blue, 0, blue.length);
     v.isNull[0] = false;
@@ -1163,7 +1471,7 @@ public class TestVectorStringExpressions
     v = new BytesColumnVector();
     v.isRepeating = false;
     v.noNulls = true;
-    
+
     // string is 2 chars long (a 3 byte and a 4 byte char)
     v.setRef(0, multiByte, 3, 7);
     batch.cols[0] = v;
@@ -1414,7 +1722,7 @@ public class TestVectorStringExpressions
         )
     );
   }
-  
+
   @Test
   public void testVectorLTrim() {
     VectorizedRowBatch b = makeTrimBatch();
@@ -1425,7 +1733,7 @@ public class TestVectorStringExpressions
         StringExpr.compare(emptyString, 0, 0, outV.vector[0], 0, 0));
     Assert.assertEquals(0,
         StringExpr.compare(blanksLeft, 2, 3, outV.vector[1], outV.start[1], outV.length[1]));
-    Assert.assertEquals(0, 
+    Assert.assertEquals(0,
         StringExpr.compare(blanksRight, 0, 5, outV.vector[2], outV.start[2], outV.length[2]));
     Assert.assertEquals(0,
         StringExpr.compare(blanksBoth, 2, 5, outV.vector[3], outV.start[3], outV.length[3]));
@@ -1445,7 +1753,7 @@ public class TestVectorStringExpressions
         StringExpr.compare(emptyString, 0, 0, outV.vector[0], 0, 0));
     Assert.assertEquals(0,
         StringExpr.compare(blanksLeft, 0, 5, outV.vector[1], outV.start[1], outV.length[1]));
-    Assert.assertEquals(0, 
+    Assert.assertEquals(0,
         StringExpr.compare(blanksRight, 0, 3, outV.vector[2], outV.start[2], outV.length[2]));
     Assert.assertEquals(0,
         StringExpr.compare(blanksBoth, 0, 5, outV.vector[3], outV.start[3], outV.length[3]));
@@ -1454,7 +1762,7 @@ public class TestVectorStringExpressions
     Assert.assertEquals(0,
         StringExpr.compare(blankString, 0, 0, outV.vector[5], outV.start[5], outV.length[5]));
   }
-  
+
   @Test
   public void testVectorTrim() {
     VectorizedRowBatch b = makeTrimBatch();
@@ -1465,7 +1773,7 @@ public class TestVectorStringExpressions
         StringExpr.compare(emptyString, 0, 0, outV.vector[0], 0, 0));
     Assert.assertEquals(0,
         StringExpr.compare(blanksLeft, 2, 3, outV.vector[1], outV.start[1], outV.length[1]));
-    Assert.assertEquals(0, 
+    Assert.assertEquals(0,
         StringExpr.compare(blanksRight, 0, 3, outV.vector[2], outV.start[2], outV.length[2]));
     Assert.assertEquals(0,
         StringExpr.compare(blanksBoth, 2, 3, outV.vector[3], outV.start[3], outV.length[3]));
@@ -1474,7 +1782,7 @@ public class TestVectorStringExpressions
     Assert.assertEquals(0,
         StringExpr.compare(blankString, 0, 0, outV.vector[5], outV.start[5], outV.length[5]));
   }
-  
+
   // Make a batch to test the trim functions.
   private VectorizedRowBatch makeTrimBatch() {
     VectorizedRowBatch b = new VectorizedRowBatch(2);