You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2016/07/07 07:05:33 UTC

[7/7] hive git commit: Revert "HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)"

Revert "HIVE-13084: Vectorization add support for PROJECTION Multi-AND/OR (Matt McCline, reviewed by Sergey Shelukhin)"

This reverts commit 0a24c8859f8a0cd6dc4dec99acbaf7f7aa93aa32.

Conflicts:
	ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/5c58dcee
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/5c58dcee
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/5c58dcee

Branch: refs/heads/master
Commit: 5c58dceeaf662b6314eedb9afa01a2896657ef77
Parents: 2233508
Author: Matt McCline <mm...@hortonworks.com>
Authored: Wed Jul 6 23:36:21 2016 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Wed Jul 6 23:36:21 2016 -0700

----------------------------------------------------------------------
 .../vectorization/VectorizedLogicBench.java     |  12 +-
 .../test/resources/testconfiguration.properties |   2 -
 .../ql/exec/vector/VectorizationContext.java    | 125 +--
 .../ql/exec/vector/expressions/ColAndCol.java   | 687 +++++-----------
 .../ql/exec/vector/expressions/ColOrCol.java    | 694 +++++-----------
 .../exec/vector/TestVectorizationContext.java   |   4 +
 .../TestVectorLogicalExpressions.java           |   4 +-
 .../vector_multi_and_projection.q               | 196 -----
 .../clientpositive/vector_multi_or_projection.q | 198 -----
 .../tez/vector_multi_and_projection.q.out       | 821 -------------------
 .../tez/vector_multi_or_projection.q.out        | 821 -------------------
 .../vector_multi_and_projection.q.out           | 800 ------------------
 .../vector_multi_or_projection.q.out            | 800 ------------------
 13 files changed, 497 insertions(+), 4667 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
----------------------------------------------------------------------
diff --git a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
index 7ff6158..50dadb2 100644
--- a/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
+++ b/itests/hive-jmh/src/main/java/org/apache/hive/benchmark/vectorization/VectorizedLogicBench.java
@@ -55,7 +55,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColAndCol(new int[] {0, 1}, 2);
+      expression = new ColAndCol(0, 1, 2);
     }
   }
 
@@ -64,7 +64,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanRepeatingLongColumnVector());
-      expression = new ColAndCol(new int[] {0, 1}, 2);
+      expression = new ColAndCol(0, 1, 2);
     }
   }
 
@@ -73,7 +73,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColAndCol(new int[] {0, 1}, 2);
+      expression = new ColAndCol(0, 1, 2);
     }
   }
 
@@ -82,7 +82,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColOrCol(new int[] {0, 1}, 2);
+      expression = new ColOrCol(0, 1, 2);
     }
   }
 
@@ -91,7 +91,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanLongColumnVector(),
           getBooleanRepeatingLongColumnVector());
-      expression = new ColOrCol(new int[] {0, 1}, 2);
+      expression = new ColOrCol(0, 1, 2);
     }
   }
 
@@ -100,7 +100,7 @@ public class VectorizedLogicBench {
     public void setup() {
       rowBatch = buildRowBatch(new LongColumnVector(), 2, getBooleanRepeatingLongColumnVector(),
           getBooleanLongColumnVector());
-      expression = new ColOrCol(new int[] {0, 1}, 2);
+      expression = new ColOrCol(0, 1, 2);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index cf44cfe..eb0d1d7 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -315,9 +315,7 @@ minitez.query.files.shared=acid_globallimit.q,\
   vector_leftsemi_mapjoin.q,\
   vector_mapjoin_reduce.q,\
   vector_mr_diff_schema_alias.q,\
-  vector_multi_and_projection.q,\
   vector_multi_insert.q,\
-  vector_multi_or_projection.q,\
   vector_non_string_partition.q,\
   vector_nullsafe_join.q,\
   vector_null_projection.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index 75c41b6..2887395 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -123,7 +123,6 @@ import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
 import org.apache.hadoop.hive.serde2.typeinfo.BaseCharTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
@@ -979,33 +978,36 @@ public class VectorizationContext {
 
     int numChildren = (childExpr == null) ? 0 : childExpr.size();
 
-    if (genericeUdf != null &&
+    if (numChildren > 2 && genericeUdf != null && mode == VectorExpressionDescriptor.Mode.FILTER &&
         ((genericeUdf instanceof GenericUDFOPOr) || (genericeUdf instanceof GenericUDFOPAnd))) {
 
-      // Special case handling for Multi-OR and Multi-AND FILTER and PROJECTION.
+      // Special case handling for Multi-OR and Multi-AND.
 
+      for (int i = 0; i < numChildren; i++) {
+        ExprNodeDesc child = childExpr.get(i);
+        String childTypeString = child.getTypeString();
+        if (childTypeString == null) {
+          throw new HiveException("Null child type name string");
+        }
+        TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(childTypeString);
+        Type columnVectorType = VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
+        if (columnVectorType != ColumnVector.Type.LONG){
+          return null;
+        }
+        if (!(child instanceof ExprNodeGenericFuncDesc) && !(child instanceof ExprNodeColumnDesc)) {
+          return null;
+        }
+      }
       Class<?> vclass;
       if (genericeUdf instanceof GenericUDFOPOr) {
-        if (mode == VectorExpressionDescriptor.Mode.PROJECTION) {
-          vclass = ColOrCol.class;
-        } else {
-          vclass = FilterExprOrExpr.class;
-        }
+        vclass = FilterExprOrExpr.class;
       } else if (genericeUdf instanceof GenericUDFOPAnd) {
-        if (mode == VectorExpressionDescriptor.Mode.PROJECTION) {
-          vclass = ColAndCol.class;
-        } else {
-          vclass = FilterExprAndExpr.class;
-        }
+        vclass = FilterExprAndExpr.class;
       } else {
         throw new RuntimeException("Unexpected multi-child UDF");
       }
       VectorExpressionDescriptor.Mode childrenMode = getChildrenMode(mode, udfClass);
-      if (mode == VectorExpressionDescriptor.Mode.PROJECTION) {
-        return createVectorMultiAndOrProjectionExpr(vclass, childExpr, childrenMode, returnType);
-      } else {
-        return createVectorExpression(vclass, childExpr, childrenMode, returnType);
-      }
+      return createVectorExpression(vclass, childExpr, childrenMode, returnType);
     }
     if (numChildren > VectorExpressionDescriptor.MAX_NUM_ARGUMENTS) {
       return null;
@@ -1044,37 +1046,6 @@ public class VectorizationContext {
     return createVectorExpression(vclass, childExpr, childrenMode, returnType);
   }
 
-  private void determineChildrenVectorExprAndArguments(Class<?> vectorClass,
-      List<ExprNodeDesc> childExpr, int numChildren, VectorExpressionDescriptor.Mode childrenMode,
-      VectorExpression.Type [] inputTypes, List<VectorExpression> children, Object[] arguments)
-          throws HiveException {
-    for (int i = 0; i < numChildren; i++) {
-      ExprNodeDesc child = childExpr.get(i);
-      String undecoratedName = getUndecoratedName(child.getTypeInfo().getTypeName());
-      inputTypes[i] = VectorExpression.Type.getValue(undecoratedName);
-      if (inputTypes[i] == VectorExpression.Type.OTHER){
-        throw new HiveException("No vector type for " + vectorClass.getSimpleName() + " argument #" + i + " type name " + undecoratedName);
-      }
-      if (child instanceof ExprNodeGenericFuncDesc) {
-        VectorExpression vChild = getVectorExpression(child, childrenMode);
-          children.add(vChild);
-          arguments[i] = vChild.getOutputColumn();
-      } else if (child instanceof ExprNodeColumnDesc) {
-        int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
-          if (childrenMode == VectorExpressionDescriptor.Mode.FILTER) {
-            // In filter mode, the column must be a boolean
-            children.add(new SelectColumnIsTrue(colIndex));
-          }
-          arguments[i] = colIndex;
-      } else if (child instanceof ExprNodeConstantDesc) {
-        Object scalarValue = getVectorTypeScalarValue((ExprNodeConstantDesc) child);
-        arguments[i] = (null == scalarValue) ? getConstantVectorExpression(null, child.getTypeInfo(), childrenMode) : scalarValue;
-      } else {
-        throw new HiveException("Cannot handle expression type: " + child.getClass().getSimpleName());
-      }
-    }
-  }
-
   private VectorExpression createVectorExpression(Class<?> vectorClass,
       List<ExprNodeDesc> childExpr, VectorExpressionDescriptor.Mode childrenMode, TypeInfo returnType) throws HiveException {
     int numChildren = childExpr == null ? 0: childExpr.size();
@@ -1082,41 +1053,31 @@ public class VectorizationContext {
     List<VectorExpression> children = new ArrayList<VectorExpression>();
     Object[] arguments = new Object[numChildren];
     try {
-      determineChildrenVectorExprAndArguments(vectorClass, childExpr, numChildren, childrenMode,
-          inputTypes, children, arguments);
-      VectorExpression  vectorExpression = instantiateExpression(vectorClass, returnType, arguments);
-      vectorExpression.setInputTypes(inputTypes);
-      if ((vectorExpression != null) && !children.isEmpty()) {
-        vectorExpression.setChildExpressions(children.toArray(new VectorExpression[0]));
-      }
-      return vectorExpression;
-    } catch (Exception ex) {
-      throw new HiveException(ex);
-    } finally {
-      for (VectorExpression ve : children) {
-        ocm.freeOutputColumn(ve.getOutputColumn());
-      }
-    }
-  }
-
-  private VectorExpression createVectorMultiAndOrProjectionExpr(Class<?> vectorClass,
-      List<ExprNodeDesc> childExpr, VectorExpressionDescriptor.Mode childrenMode, TypeInfo returnType) throws HiveException {
-    int numChildren = childExpr == null ? 0: childExpr.size();
-    VectorExpression.Type [] inputTypes = new VectorExpression.Type[numChildren];
-    List<VectorExpression> children = new ArrayList<VectorExpression>();
-    Object[] arguments = new Object[numChildren];
-    try {
-      determineChildrenVectorExprAndArguments(vectorClass, childExpr, numChildren, childrenMode,
-          inputTypes, children, arguments);
-
-      // For Multi-AND/OR, transform the arguments -- column indices into an array of int.
-      int[] colNums = new int[numChildren];
       for (int i = 0; i < numChildren; i++) {
-        colNums[i] = (Integer) arguments[i];
+        ExprNodeDesc child = childExpr.get(i);
+        String undecoratedName = getUndecoratedName(child.getTypeInfo().getTypeName());
+        inputTypes[i] = VectorExpression.Type.getValue(undecoratedName);
+        if (inputTypes[i] == VectorExpression.Type.OTHER){
+          throw new HiveException("No vector type for " + vectorClass.getSimpleName() + " argument #" + i + " type name " + undecoratedName);
+        }
+        if (child instanceof ExprNodeGenericFuncDesc) {
+          VectorExpression vChild = getVectorExpression(child, childrenMode);
+            children.add(vChild);
+            arguments[i] = vChild.getOutputColumn();
+        } else if (child instanceof ExprNodeColumnDesc) {
+          int colIndex = getInputColumnIndex((ExprNodeColumnDesc) child);
+            if (childrenMode == VectorExpressionDescriptor.Mode.FILTER) {
+              // In filter mode, the column must be a boolean
+              children.add(new SelectColumnIsTrue(colIndex));
+            }
+            arguments[i] = colIndex;
+        } else if (child instanceof ExprNodeConstantDesc) {
+          Object scalarValue = getVectorTypeScalarValue((ExprNodeConstantDesc) child);
+          arguments[i] = (null == scalarValue) ? getConstantVectorExpression(null, child.getTypeInfo(), childrenMode) : scalarValue;
+        } else {
+          throw new HiveException("Cannot handle expression type: " + child.getClass().getSimpleName());
+        }
       }
-      arguments = new Object[1];
-      arguments[0] = colNums;
-
       VectorExpression  vectorExpression = instantiateExpression(vectorClass, returnType, arguments);
       vectorExpression.setInputTypes(inputTypes);
       if ((vectorExpression != null) && !children.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
index 48e3070..ff7371d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColAndCol.java
@@ -17,38 +17,26 @@
  */
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-import java.util.Arrays;
-
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 
-import com.google.common.base.Preconditions;
-
 /**
- * Evaluate AND of 2 or more boolean columns and store the boolean result in the
- * output boolean column.  This is a projection or result producing expression (as opposed to
- * a filter expression).
- *
- * Some child boolean columns may be vector expressions evaluated into boolean scratch columns.
+ * Evaluate AND of two boolean columns and store result in the output boolean column.
  */
 public class ColAndCol extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private int[] colNums;
+  private int colNum1;
+  private int colNum2;
   private int outputColumn;
-  private int[] mapToChildExpression;
-  private int[] andSelected;
-  private boolean[] intermediateNulls;
 
-  public ColAndCol(int[] colNums, int outputColumn) {
+  public ColAndCol(int colNum1, int colNum2, int outputColumn) {
     this();
-    this.colNums = colNums;
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
     this.outputColumn = outputColumn;
-    mapToChildExpression = null;
-    andSelected = new int[VectorizedRowBatch.DEFAULT_SIZE];
-    intermediateNulls = new boolean[VectorizedRowBatch.DEFAULT_SIZE];
   }
 
   public ColAndCol() {
@@ -58,493 +46,240 @@ public class ColAndCol extends VectorExpression {
   @Override
   public void evaluate(VectorizedRowBatch batch) {
 
-    Preconditions.checkState(colNums.length >= 2);
-
-    /*
-     * Vector child expressions will be omitted if they are existing boolean vector columns,
-     * so the child index does not necessarily index into the childExpressions.
-     * We construct a simple index map to the child expression in mapToChildExpression.
-     */
-    if (childExpressions != null && mapToChildExpression == null) {
-      // 
-      mapToChildExpression = new int [colNums.length];
-      int childIndex = 0;
-      for (int i = 0; i < childExpressions.length; i++) {
-        VectorExpression ve = childExpressions[i];
-        int outputColumn = ve.getOutputColumn();
-        while (outputColumn != colNums[childIndex]) {
-          mapToChildExpression[childIndex++] = -1;
-        }
-        mapToChildExpression[childIndex++] = i;
-      }
-      Preconditions.checkState(childIndex == colNums.length);
-    }
-
-    final int n = batch.size;
-    if (n <= 0) {
-      // Nothing to do
-      return;
-    }
-
     if (childExpressions != null) {
-      /*
-       * Evaluate first child expression.  Other child are conditionally evaluated later
-       * based on whether there is still a need for AND processing.
-       */
-      int childExpressionIndex = mapToChildExpression[0];
-      if (childExpressionIndex != -1) {
-        VectorExpression ve = childExpressions[childExpressionIndex];
-        Preconditions.checkState(ve.getOutputColumn() == colNums[0]);
-        ve.evaluate(batch);
-      }
+      super.evaluateChildren(batch);
     }
 
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
     int[] sel = batch.selected;
+    int n = batch.size;
+    long[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
 
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     long[] outputVector = outV.vector;
+    if (n <= 0) {
+      // Nothing to do
+      return;
+    }
 
-    /**
-     * Null processing complicates the algorithm here for Multi-AND.
-     *
-     * All true --> true
-     * 0 or more true with 1 or more null --> result = null
-     * Any false --> false
-     *
-     * For AND-processing, we remember nulls in the intermediateNulls array as we go along so
-     * later we can mark the row as null instead of true when there is a null.
-     */
-
-    /*
-     * andRepeating will be true when all the children column vectors processed so far are
-     * some combination of repeating true and repeating null.
-     * andRepeatingIsNull will be true when there has been at least one repeating null column.
-     */
-    boolean andRepeating = false;
-    boolean andRepeatingIsNull = false;
-
-    /*
-     * The andSel variable and andSelected member array represent rows that have at have
-     * some combination of true and nulls.
-     */
-    int andSel = 0;
-
-    Arrays.fill(intermediateNulls, 0, VectorizedRowBatch.DEFAULT_SIZE, false);
-
-    // Reset noNulls to true, isNull to false, isRepeating to false.
-    outV.reset();
-
-    LongColumnVector firstColVector = (LongColumnVector) batch.cols[colNums[0]];
-    long[] firstVector = firstColVector.vector;
-
-    /*
-     * We prime the pump by evaluating the first child to see if we are starting with
-     * andRepeating/andRepeatingHasNulls or we are starting with andSel/andSelected processing.
-     */
-    if (firstColVector.isRepeating) {
-      if (firstColVector.noNulls || !firstColVector.isNull[0]) {
-        if (firstVector[0] == 0) {
-          // When the entire child column is repeating false, we are done for AND.
-          outV.isRepeating = true;
-          outputVector[0] = 0;
-          return;
+    long vector1Value = vector1[0];
+    long vector2Value = vector2[0];
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] & vector2[0];
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value & vector2[i];
+          }
         } else {
-          // First column is repeating true.
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value & vector2[i];
+          }
         }
-      } else {
-        Preconditions.checkState(firstColVector.isNull[0]);
-
-        // At least one repeating null column.
-        andRepeatingIsNull = true;
-      }
-      andRepeating = true;
-    } else if (firstColVector.noNulls) {
-
-      /*
-       * No nulls -- so all true rows go in andSel/andSelected.
-       */
-      if (batch.selectedInUse) {
-        for (int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (firstVector[i] == 1) {
-            andSelected[andSel++] = i;
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2Value;
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2Value;
           }
         }
-      } else {
-        for (int i = 0; i != n; i++) {
-          if (firstVector[i] == 1) {
-            andSelected[andSel++] = i;
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2[i];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2[i];
           }
         }
+        outV.isRepeating = false;
       }
-    } else  {
-
-      /*
-       * Can be nulls -- so all true rows and null rows go in andSel/andSelected.
-       * Remember nulls in our separate intermediateNulls array.
-       */
-      if (batch.selectedInUse) {
-        for (int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (firstColVector.isNull[i]) {
-            intermediateNulls[i] = true;
-            andSelected[andSel++] = i;
-          } else if (firstVector[i] == 1) {
-            andSelected[andSel++] = i;
+      outV.noNulls = true;
+    } else if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      // only input 2 side has nulls
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] & vector2[0];
+        outV.isNull[0] = (vector1[0] == 1) && inputColVector2.isNull[0];
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value & vector2[i];
+            outV.isNull[i] = (vector1[0] == 1) && inputColVector2.isNull[i];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value & vector2[i];
+            outV.isNull[i] = (vector1[0] == 1) && inputColVector2.isNull[i];
           }
         }
-      } else {
-        for (int i = 0; i != n; i++) {
-          if (firstColVector.isNull[i]) {
-            intermediateNulls[i] = true;
-            andSelected[andSel++] = i;
-          } else if (firstVector[i] == 1) {
-            andSelected[andSel++] = i;
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2Value;
+            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[0];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2Value;
+            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[0];
           }
         }
-      }
-    }
-
-    /*
-     * Process child #2 and above.
-     */
-    int colNum = 1;
-    do {
-      if (!andRepeating && andSel == 0) {
-
-        /*
-         * Since andSel/andSelected represent currently true entries and there are none,
-         * then nothing is true (how philosophical!).
-         */
-        break;
-      }
-
-      if (childExpressions != null) {
-        int childExpressionIndex = mapToChildExpression[colNum];
-        if (childExpressionIndex != -1) {
-          if (andRepeating) {
-
-            /*
-             * We need to start with a full evaluate on all [selected] rows.
-             */
-            VectorExpression ve = childExpressions[childExpressionIndex];
-            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
-            ve.evaluate(batch);
-          } else {
-
-            /*
-             * Evaluate next child expression.
-             * But only evaluate the andSelected rows (i.e. current true or true with nulls rows).
-             */
-            boolean saveSelectedInUse = batch.selectedInUse;
-            int[] saveSelected = sel;
-            batch.selectedInUse = true;
-            batch.selected = andSelected;
-
-            VectorExpression ve = childExpressions[childExpressionIndex];
-            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
-            ve.evaluate(batch);
-
-            batch.selectedInUse = saveSelectedInUse;
-            batch.selected = saveSelected;
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2[i];
+            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[i];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2[i];
+            outV.isNull[i] = (vector1[i] == 1) && inputColVector2.isNull[i];
           }
         }
+        outV.isRepeating = false;
       }
-
-      LongColumnVector nextColVector = (LongColumnVector) batch.cols[colNums[colNum]];
-      long[] nextVector = nextColVector.vector;
-
-      if (andRepeating) {
-
-        /*
-         * The andRepeating flag means the whole batch is repeating true possibly with
-         * some repeating nulls.
-         */
-        if (nextColVector.isRepeating) {
-
-          /*
-           * Current child column is repeating so stay in repeating mode.
-           */
-          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
-            if (nextVector[0] == 0) {
-              // When the entire child column is repeating false, we are done for AND.
-              outV.isRepeating = true;
-              outputVector[0] = 0;
-              return;
-            } else {
-              // Current column is repeating true.
-            }
-          } else {
-            Preconditions.checkState(nextColVector.isNull[0]);
-
-            // At least one repeating null column.
-            andRepeatingIsNull = true;
+      outV.noNulls = false;
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      // only input 1 side has nulls
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] & vector2[0];
+        outV.isNull[0] = inputColVector1.isNull[0] && (vector2[0] == 1);
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value & vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 1);
           }
-          // Continue with andRepeating as true.
         } else {
-
-          /*
-           * Switch away from andRepeating/andRepeatingIsNull and now represent individual rows in
-           * andSel/andSelected.
-           */
-          if (nextColVector.noNulls) {
-
-            /*
-             * Current child column has no nulls.
-             */
-
-            Preconditions.checkState(andSel == 0);
-            andRepeating = false;
-  
-            if (andRepeatingIsNull) {
-  
-              /*
-               * Since andRepeatingIsNull is true, we always set intermediateNulls when building
-               * andSel/andSelected when the next row is true.
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextVector[i] == 1) {
-                    intermediateNulls[i] = true;
-                    andSelected[andSel++] = i;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextVector[i] == 1) {
-                    intermediateNulls[i] = true;
-                    andSelected[andSel++] = i;
-                  }
-                }
-              }
-              andRepeatingIsNull = false;
-            } else {
-  
-              /*
-               * Previous rounds were all true with no null child columns.  Just build
-               * andSel/andSelected when the next row is true.
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextVector[i] == 1) {
-                    andSelected[andSel++] = i;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextVector[i] == 1) {
-                    andSelected[andSel++] = i;
-                  }
-                }
-              }
-            }
-          } else {
-
-            /*
-             * Current child column can have nulls.
-             */
-
-            Preconditions.checkState(andSel == 0);
-            andRepeating = false;
-
-            if (andRepeatingIsNull) {
-
-              /*
-               * Since andRepeatingIsNull is true, we always set intermediateNulls when building
-               * andSel/andSelected when the next row is null or true...
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextColVector.isNull[i] || nextVector[i] == 1) {
-                    intermediateNulls[i] = true;
-                    andSelected[andSel++] = i;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextColVector.isNull[i] || nextVector[i] == 1) {
-                    intermediateNulls[i] = true;
-                    andSelected[andSel++] = i;
-                  }
-                }
-              }
-              andRepeatingIsNull = false;
-            } else {
-
-              /*
-               * Previous rounds were all true with no null child columns.  Build
-               * andSel/andSelected when the next row is true; also build when next is null
-               * and set intermediateNulls to true, too.
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextColVector.isNull[i]) {
-                    intermediateNulls[i] = true;
-                    andSelected[andSel++] = i;
-                  } else if (nextVector[i] == 1) {
-                    andSelected[andSel++] = i;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextColVector.isNull[i]) {
-                    intermediateNulls[i] = true;
-                    andSelected[andSel++] = i;
-                  } else if (nextVector[i] == 1) {
-                    andSelected[andSel++] = i;
-                  }
-                }
-              }
-            }
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value & vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 1);
           }
         }
-      } else {
-
-        /*
-         * Continue in row mode: the andSel variable and andSelected member array contains the
-         * rows that are some combination of true and null.
-         */
-        if (nextColVector.isRepeating) {
-
-          /*
-           * Current child column is repeating which affects all rows.
-           */
-          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
-
-            if (nextVector[0] == 0) {
-              // When the entire child column is repeating false, we are done for AND.
-              outV.isRepeating = true;
-              outputVector[0] = 0;
-              return;
-            } else {
-              // Child column is all true. Keep current andSel/andSelected rows.
-            }
-          } else {
-            Preconditions.checkState(nextColVector.isNull[0]);
-
-            // Column is repeating null -- need to mark all current rows in andSel/andSelected
-            // as null.
-            for (int j = 0; j < andSel; j++) {
-              int i = andSelected[j];
-              intermediateNulls[i] = true;
-            }
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2Value;
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[0] == 1);
           }
-        } else if (nextColVector.noNulls) {
-
-          /*
-           * Current child column has no nulls.
-           */
-
-          /*
-           * Rebuild andSel/andSelected to keep true rows.
-           */
-          int newSel = 0;
-          for (int j = 0; j < andSel; j++) {
-            int i = andSelected[j];
-            if (nextVector[i] == 1) {
-              andSelected[newSel++] = i;
-            }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2Value;
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[0] == 1);
+          }
+        }
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 1);
           }
-          andSel = newSel;
         } else {
-
-          /*
-           * Current child column can have nulls.
-           */
-
-          /*
-           * Rebuild andSel/andSelected to keep true rows or null rows.
-           */
-          int newSel = 0;
-          for (int j = 0; j < andSel; j++) {
-            int i = andSelected[j];
-            if (nextColVector.isNull[i]) {
-              // At least one null.
-              intermediateNulls[i] = true;
-              andSelected[newSel++] = i;
-            } else if (nextVector[i] == 1) {
-              andSelected[newSel++] = i;
-            }
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 1);
           }
-          andSel = newSel;
         }
+        outV.isRepeating = false;
       }
-    } while (++colNum < colNums.length);
-
-    /*
-     * Produce final result.
-     */
-    if (andRepeating) {
-      outV.isRepeating = true;
-      if (andRepeatingIsNull) {
-        // The appearance of a null makes the repeated result null.
-        outV.noNulls = false;
-        outV.isNull[0] = true;
-      } else {
-        // All columns are repeating true.
-        outputVector[0] = 1;
-      }
-    } else if (andSel == 0) {
-      // No rows had true.
-      outV.isRepeating = true;
-      outputVector[0] = 0;
-    } else {
-      // Ok, rows were some combination of true and null throughout.
-      int andIndex = 0;
-      if (batch.selectedInUse) {
-        /*
-         * The batch selected array has all the rows we are projecting a boolean from.
-         * The andSelected array has a subset of the selected rows that have at least
-         * one true and may have some nulls. Now we need to decide if we are going to mark
-         * those rows as true, or null because there was at least one null.
-         *
-         * We use the andIndex to progress through the andSelected array and make a decision
-         * on how to fill out the boolean result.
-         *
-         * Since we reset the output column, we shouldn't have to set isNull false for true
-         * entries.
-         */
-        for (int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (andIndex < andSel && andSelected[andIndex] == i) {
-            // We haven't processed all the andSelected entries and the row index is in
-            // andSelected, so make a result decision for true or null.
-            if (intermediateNulls[i]) {
-              outV.noNulls = false;
-              outV.isNull[i] = true;
-            } else {
-              outputVector[i] = 1;
-            }
-            andIndex++;
-          } else {
-            // The row is not in the andSelected array.  Result is false.
-            outputVector[i] = 0;
+      outV.noNulls = false;
+    } else /* !inputColVector1.noNulls && !inputColVector2.noNulls */{
+      // either input 1 or input 2 may have nulls
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] & vector2[0];
+        outV.isNull[0] = ((vector1[0] == 1) && inputColVector2.isNull[0])
+            || (inputColVector1.isNull[0] && (vector2[0] == 1))
+            || (inputColVector1.isNull[0] && inputColVector2.isNull[0]);
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value & vector2[i];
+            outV.isNull[i] = ((vector1[0] == 1) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[0] && (vector2[i] == 1))
+                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value & vector2[i];
+            outV.isNull[i] = ((vector1[0] == 1) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[0] && (vector2[i] == 1))
+                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
           }
         }
-        Preconditions.checkState(andIndex == andSel);
-      } else {
-        /*
-         * The andSelected array has a subset of the selected rows that have at least
-         * one true and may have some nulls. Now we need to decide if we are going to mark
-         * those rows as true, or null because there was at least one null.
-         *
-         * Prefill the result as all false.  Then decide about the andSelected entries.
-         */
-        Arrays.fill(outputVector, 0, n, 0);
-        for (int j = 0; j < andSel; j++) {
-          int i = andSelected[j];
-          if (intermediateNulls[i]) {
-            outV.noNulls = false;
-            outV.isNull[i] = true;
-          } else {
-            outputVector[i] = 1;
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2Value;
+            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[0])
+                || (inputColVector1.isNull[i] && (vector2[0] == 1))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2Value;
+            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[0])
+                || (inputColVector1.isNull[i] && (vector2[0] == 1))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
           }
         }
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] & vector2[i];
+            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[i] && (vector2[i] == 1))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] & vector2[i];
+            outV.isNull[i] = ((vector1[i] == 1) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[i] && (vector2[i] == 1))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
+          }
+        }
+        outV.isRepeating = false;
       }
+      outV.noNulls = false;
     }
   }
 
@@ -558,6 +293,22 @@ public class ColAndCol extends VectorExpression {
     return "boolean";
   }
 
+  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;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
index db33281..60ed2d4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/ColOrCol.java
@@ -18,38 +18,28 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-import java.util.Arrays;
-
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 
-import com.google.common.base.Preconditions;
-
 /**
- * Evaluate OR of 2 or more boolean columns and store the boolean result in the
- * output boolean column.  This is a projection or result producing expression (as opposed to
- * a filter expression).
- *
- * Some child boolean columns may be vector expressions evaluated into boolean scratch columns.
+ * This class performs OR expression on two input columns and stores,
+ * the boolean output in a separate output column. The boolean values
+ * are supposed to be represented as 0/1 in a long vector.
  */
 public class ColOrCol extends VectorExpression {
 
   private static final long serialVersionUID = 1L;
 
-  private int[] colNums;
+  private int colNum1;
+  private int colNum2;
   private int outputColumn;
-  private int[] mapToChildExpression;
-  private int[] orSelected;
-  private boolean[] intermediateNulls;
 
-  public ColOrCol(int[] colNums, int outputColumn) {
+  public ColOrCol(int colNum1, int colNum2, int outputColumn) {
     this();
-    this.colNums = colNums;
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
     this.outputColumn = outputColumn;
-    mapToChildExpression = null;
-    orSelected = new int[VectorizedRowBatch.DEFAULT_SIZE];
-    intermediateNulls = new boolean[VectorizedRowBatch.DEFAULT_SIZE];
   }
 
   public ColOrCol() {
@@ -59,494 +49,240 @@ public class ColOrCol extends VectorExpression {
   @Override
   public void evaluate(VectorizedRowBatch batch) {
 
-    Preconditions.checkState(colNums.length >= 2);
-
-    /*
-     * Vector child expressions will be omitted if they are existing boolean vector columns,
-     * so the child index does not necessarily index into the childExpressions.
-     * We construct a simple index map to the child expression in mapToChildExpression.
-     */
-    if (childExpressions != null && mapToChildExpression == null) {
-      // Some vector child expressions can be omitted (e.g. if they are existing boolean columns).
-      mapToChildExpression = new int [colNums.length];
-      int childIndex = 0;
-      for (int i = 0; i < childExpressions.length; i++) {
-        VectorExpression ve = childExpressions[i];
-        int outputColumn = ve.getOutputColumn();
-        while (outputColumn != colNums[childIndex]) {
-          mapToChildExpression[childIndex++] = -1;
-        }
-        mapToChildExpression[childIndex++] = i;
-      }
-      Preconditions.checkState(childIndex == colNums.length);
-    }
-
-    final int n = batch.size;
-    if (n <= 0) {
-      // Nothing to do
-      return;
-    }
-
     if (childExpressions != null) {
-      /*
-       * Evaluate first child expression.  Other child are conditionally evaluated later
-       * based on whether there is still a need for OR processing.
-       */
-      int childExpressionIndex = mapToChildExpression[0];
-      if (childExpressionIndex != -1) {
-        VectorExpression ve = childExpressions[childExpressionIndex];
-        Preconditions.checkState(ve.getOutputColumn() == colNums[0]);
-        ve.evaluate(batch);
-      }
+      super.evaluateChildren(batch);
     }
 
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
     int[] sel = batch.selected;
+    int n = batch.size;
+    long[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
 
     LongColumnVector outV = (LongColumnVector) batch.cols[outputColumn];
     long[] outputVector = outV.vector;
-
-    /**
-     * Null processing complicates the algorithm here for Multi-OR.
-     *
-     * Any true --> true
-     * 0 or more false with 1 or more null --> result = null
-     * All false --> false
-     *
-     * For OR processing, we set the outputVector row to true rows as we find them.
-     * Once set for OR, the row result doesn't change.
-     *
-     * We remember nulls in the intermediateNulls so we can later we can mark the row as null
-     * instead of false if there was a null.
-     */
-
-    /*
-     * orRepeating will be true when all the children column vectors processed so far are
-     * some combination of repeating false and repeating null.
-     * orRepeatingIsNull will be true when there is at least one repeating null column.
-     */
-    boolean orRepeating = false;
-    boolean orRepeatingHasNulls = false;
-
-    /*
-     * The orSel variable and orSelected member array represent rows that have at have
-     * some combination of false and nulls.
-     */
-    int orSel = 0;
-
-    Arrays.fill(intermediateNulls, 0, VectorizedRowBatch.DEFAULT_SIZE, false);
-
-    // Reset noNulls to true, isNull to false, isRepeating to false.
-    outV.reset();
-
-    // Initially, set all rows to false.
-    if (batch.selectedInUse) {
-      for (int j = 0; j != n; j++) {
-        int i = sel[j];
-        outputVector[i] = 0;
-      }
-    } else {
-      Arrays.fill(outputVector, 0, n, 0);
+    if (n <= 0) {
+      // Nothing to do
+      return;
     }
 
-    LongColumnVector firstColVector = (LongColumnVector) batch.cols[colNums[0]];
-    long[] firstVector = firstColVector.vector;
-
-    /*
-     * We prime the pump by evaluating the first child to see if we are starting with
-     * orRepeating/orRepeatingHasNulls or we are starting with orSel/orSelected processing.
-     */
-    if (firstColVector.isRepeating) {
-      if (firstColVector.noNulls || !firstColVector.isNull[0]) {
-        if (firstVector[0] == 1) {
-          // When the entire child column is repeating true, we are done for OR.
-          outV.isRepeating = true;
-          outputVector[0] = 1;
-          return;
+    long vector1Value = vector1[0];
+    long vector2Value = vector2[0];
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] | vector2[0];
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value | vector2[i];
+          }
         } else {
-          // First column is repeating false.
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value | vector2[i];
+          }
         }
-      } else {
-        Preconditions.checkState(firstColVector.isNull[0]);
-
-        // At least one repeating null column.
-        orRepeatingHasNulls = true;
-      }
-      orRepeating = true;
-    } else if (firstColVector.noNulls) {
-
-      /*
-       * No nulls -- so all false rows go in orSel/orSelected.  Otherwise, when the row is true,
-       * mark the output row as true.
-       */
-      if (batch.selectedInUse) {
-        for (int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (firstVector[i] == 0) {
-            orSelected[orSel++] = i;
-          } else {
-            outputVector[i] = 1;
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2Value;
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2Value;
           }
         }
-      } else {
-        for (int i = 0; i != n; i++) {
-          if (firstVector[i] == 0) {
-            orSelected[orSel++] = i;
-          } else {
-            outputVector[i] = 1;
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2[i];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2[i];
           }
         }
+        outV.isRepeating = false;
       }
-    } else {
-
-      /*
-       * Can be nulls -- so all false rows and null rows go in orSel/orSelected.
-       * Remember nulls in our separate intermediateNulls array.  Otherwise, when the row is true,
-       * mark the output row as true.
-       */
-      if (batch.selectedInUse) {
-        for (int j = 0; j != n; j++) {
-          int i = sel[j];
-          if (firstColVector.isNull[i]) {
-            intermediateNulls[i] = true;
-            orSelected[orSel++] = i;
-          } else if (firstVector[i] == 0) {
-            orSelected[orSel++] = i;
-          } else {
-            outputVector[i] = 1;
+      outV.noNulls = true;
+    } else if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      // only input 2 side has nulls
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] | vector2[0];
+        outV.isNull[0] = (vector1[0] == 0) && inputColVector2.isNull[0];
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value | vector2[i];
+            outV.isNull[i] = (vector1Value == 0) && inputColVector2.isNull[i];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value | vector2[i];
+            outV.isNull[i] = (vector1Value == 0) && inputColVector2.isNull[i];
           }
         }
-      } else {
-        for (int i = 0; i != n; i++) {
-          if (firstColVector.isNull[i]) {
-            intermediateNulls[i] = true;
-            orSelected[orSel++] = i;
-          } else if (firstVector[i] == 0) {
-            orSelected[orSel++] = i;
-          } else {
-            outputVector[i] = 1;
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2Value;
+            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[0];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2Value;
+            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[0];
           }
         }
-      }
-    }
-
-    /*
-     * Process child #2 and above.
-     */
-    int colNum = 1;
-    do {
-      if (!orRepeating && orSel == 0) {
-
-        /*
-         * Since orSelected and orSel represent currently false entries and there are none,
-         * then everything is true (how philosophical!).
-         */
-        break;
-      }
-
-      if (childExpressions != null) {
-        int childExpressionIndex = mapToChildExpression[colNum];
-        if (childExpressionIndex != -1) {
-          if (orRepeating) {
-            /*
-             * We need to start with a full evaluate on all [selected] rows.
-             */
-            VectorExpression ve = childExpressions[childExpressionIndex];
-            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
-            ve.evaluate(batch);
-          } else {
-            /*
-             * Evaluate next child expression.
-             * But only on the orSelected rows (i.e. current false or false with nulls rows).
-             */
-            boolean saveSelectedInUse = batch.selectedInUse;
-            int[] saveSelected = sel;
-            batch.selectedInUse = true;
-            batch.selected = orSelected;
-
-            VectorExpression ve = childExpressions[childExpressionIndex];
-            Preconditions.checkState(ve.getOutputColumn() == colNums[colNum]);
-            ve.evaluate(batch);
-
-            batch.selectedInUse = saveSelectedInUse;
-            batch.selected = saveSelected;
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2[i];
+            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[i];
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2[i];
+            outV.isNull[i] = (vector1[i] == 0) && inputColVector2.isNull[i];
           }
         }
+        outV.isRepeating = false;
       }
-
-      LongColumnVector nextColVector = (LongColumnVector) batch.cols[colNums[colNum]];
-      long[] nextVector = nextColVector.vector;
-
-      if (orRepeating) {
-
-        /*
-         * The orRepeating flag means the whole batch has some combination of repeating false
-         * columns and repeating null columns.
-         */
-        if (nextColVector.isRepeating) {
-
-          /*
-           * Current child column is repeating so stay in repeating mode.
-           */
-          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
-            if (nextVector[0] == 1) {
-              outV.isRepeating = true;
-              outputVector[0] = 1;
-              return;
-            } else {
-              // The orRepeatingHasNulls flag goes on to the next stage, too.
-            }
-          } else {
-            Preconditions.checkState(nextColVector.isNull[0]);
-
-            // At least one repeating null column.
-            orRepeatingHasNulls = true;
+      outV.noNulls = false;
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      // only input 1 side has nulls
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] | vector2[0];
+        outV.isNull[0] = inputColVector1.isNull[0] && (vector2[0] == 0);
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value | vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 0);
           }
-          // Continue with orRepeating as true.
         } else {
-
-          /*
-           * Switch away from orRepeating/orRepeatingHasNulls and now represent individual rows in
-           * orSel/orSelected.
-           */
-          if (nextColVector.noNulls) {
-
-            /*
-             * Current child column has no nulls.
-             */
-
-            Preconditions.checkState(orSel == 0);
-            orRepeating = false;
-  
-            if (orRepeatingHasNulls) {
-
-              /*
-               * Since orRepeatingIsNull is true, we always set intermediateNulls when building
-               * orSel/orSelected when the next row is false.  Otherwise, when the row is true, mark
-               * the output row as true.
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextVector[i] == 0) {
-                    intermediateNulls[i] = true;
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextVector[i] == 0) {
-                    intermediateNulls[i] = true;
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              }
-              orRepeatingHasNulls = false;
-            } else {
-  
-              /*
-               * Previous rounds were all false with no null child columns.  Build
-               * orSel/orSelected when the next row is false.  Otherwise, when the row is true, mark
-               * the output row as true.
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextVector[i] == 0) {
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextVector[i] == 0) {
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              }
-            }
-          } else {
-
-            /*
-             * Current child column can have nulls.
-             */
-
-            Preconditions.checkState(orSel == 0);
-            orRepeating = false;
-  
-            if (orRepeatingHasNulls) {
-  
-              /*
-               * Since orRepeatingIsNull is true, we always set intermediateNulls when building
-               * orSel/orSelected when the next row is null or false.  Otherwise, when the row
-               * is true mark the output row as true.
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextColVector.isNull[i] || nextVector[i] == 0) {
-                    intermediateNulls[i] = true;
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextColVector.isNull[i] || nextVector[i] == 0) {
-                    intermediateNulls[i] = true;
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              }
-              orRepeatingHasNulls = false;
-            } else {
-  
-              /*
-               * Previous rounds were all true with no null child columns.  Build
-               * andSel/andSelected when the next row is true; also build when next is null
-               * and set intermediateNulls to true, too.  Otherwise, when the row
-               * is true mark the output row as true.
-               */
-              if (batch.selectedInUse) {
-                for (int j = 0; j != n; j++) {
-                  int i = sel[j];
-                  if (nextColVector.isNull[i]) {
-                    intermediateNulls[i] = true;
-                    orSelected[orSel++] = i;
-                  } else if (nextVector[i] == 0) {
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              } else {
-                for (int i = 0; i != n; i++) {
-                  if (nextColVector.isNull[i]) {
-                    intermediateNulls[i] = true;
-                    orSelected[orSel++] = i;
-                  } else if (nextVector[i] == 0) {
-                    orSelected[orSel++] = i;
-                  } else {
-                    outputVector[i] = 1;
-                  }
-                }
-              }
-            }
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value | vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[0] && (vector2[i] == 0);
           }
         }
-      } else {
-
-        /*
-         * Continue in row mode: the orSel variable and orSelected member array contains the
-         * rows that are a combination of false and null.
-         */
-        if (nextColVector.isRepeating) {
-
-          if (nextColVector.noNulls || !nextColVector.isNull[0]) {
-
-            if (nextVector[0] == 1) {
-              // When the entire child column is repeating true, we are done for OR.
-              outV.isRepeating = true;
-              outputVector[0] = 1;
-              return;
-            } else {
-              // Child column is all false. Keep all orSel/orSelected rows.
-            }
-          } else {
-            Preconditions.checkState(nextColVector.isNull[0]);
-
-            // Column is repeating null -- need to mark all current rows in orSel/orSelected
-            // as null.
-            for (int j = 0; j < orSel; j++) {
-              int i = orSelected[j];
-              intermediateNulls[i] = true;
-            }
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2Value;
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2Value == 0);
           }
-        } else if (nextColVector.noNulls) {
-
-          /*
-           * Current child column has no nulls.
-           */
-
-          /*
-           * Rebuild orSel/orSelected to keep false rows.  True rows get set in output vector.
-           */
-          int newSel = 0;
-          for (int j = 0; j < orSel; j++) {
-            int i = orSelected[j];
-            if (nextVector[i] == 0) {
-              orSelected[newSel++] = i;
-            } else {
-              outputVector[i] = 1;
-            }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2Value;
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2Value == 0);
+          }
+        }
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 0);
           }
-          orSel = newSel;
         } else {
-
-          /*
-           * Current child column can have nulls.
-           */
-
-          /*
-           * Rebuild orSel/orSelected to keep false rows or null rows.  True rows get set in
-           * output vector.
-           */
-          int newSel = 0;
-          for (int j = 0; j < orSel; j++) {
-            int i = orSelected[j];
-            if (nextColVector.isNull[i]) {
-              // Mark row has at least one null.
-              intermediateNulls[i] = true;
-              orSelected[newSel++] = i;
-            } else if (nextVector[i] == 0) {
-              orSelected[newSel++] = i;
-            } else {
-              outputVector[i] = 1;
-            }
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2[i];
+            outV.isNull[i] = inputColVector1.isNull[i] && (vector2[i] == 0);
           }
-          orSel = newSel;
         }
+        outV.isRepeating = false;
       }
-    } while (++colNum < colNums.length);
-
-    /*
-     *  Produce final result.
-     */
-    if (orRepeating) {
-
-      /*
-       * The orRepeating flags means the whole batch is false and may have nulls.
-       */
-      outV.isRepeating = true;
-      if (orRepeatingHasNulls) {
-        outV.noNulls = false;
-        outV.isNull[0] = true;
-      } else {
-        outputVector[0] = 0;
-      }
-    } else {
-
-      /*
-       * When were any intermediate nulls for a row, the result row will be null.
-       * Note the true entries were previously set in outputVector as we went along.
-       */
-      for (int j = 0; j < orSel; j++) {
-        int i = orSelected[j];
-        Preconditions.checkState(outputVector[i] == 0);
-        if (intermediateNulls[i]) {
-          outV.noNulls = false;
-          outV.isNull[i] = true;
+      outV.noNulls = false;
+    } else /* !inputColVector1.noNulls && !inputColVector2.noNulls */{
+      // either input 1 or input 2 may have nulls
+      if ((inputColVector1.isRepeating) && (inputColVector2.isRepeating)) {
+        // All must be selected otherwise size would be zero
+        // Repeating property will not change.
+        outV.isRepeating = true;
+        outputVector[0] = vector1[0] | vector2[0];
+        outV.isNull[0] = ((vector1[0] == 0) && inputColVector2.isNull[0])
+            || (inputColVector1.isNull[0] && (vector2[0] == 0))
+            || (inputColVector1.isNull[0] && inputColVector2.isNull[0]);
+      } else if (inputColVector1.isRepeating && !inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1Value | vector2[i];
+            outV.isNull[i] = ((vector1[0] == 0) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[0] && (vector2[i] == 0))
+                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1Value | vector2[i];
+            outV.isNull[i] = ((vector1[0] == 0) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[0] && (vector2[i] == 0))
+                || (inputColVector1.isNull[0] && inputColVector2.isNull[i]);
+          }
+        }
+        outV.isRepeating = false;
+      } else if (!inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2Value;
+            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[0])
+                || (inputColVector1.isNull[i] && (vector2[0] == 0))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2Value;
+            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[0])
+                || (inputColVector1.isNull[i] && (vector2[0] == 0))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[0]);
+          }
         }
+        outV.isRepeating = false;
+      } else /* neither side is repeating */{
+        if (batch.selectedInUse) {
+          for (int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputVector[i] = vector1[i] | vector2[i];
+            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[i] && (vector2[i] == 0))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
+          }
+        } else {
+          for (int i = 0; i != n; i++) {
+            outputVector[i] = vector1[i] | vector2[i];
+            outV.isNull[i] = ((vector1[i] == 0) && inputColVector2.isNull[i])
+                || (inputColVector1.isNull[i] && (vector2[i] == 0))
+                || (inputColVector1.isNull[i] && inputColVector2.isNull[i]);
+          }
+        }
+        outV.isRepeating = false;
       }
+      outV.noNulls = false;
     }
   }
 
@@ -560,6 +296,22 @@ public class ColOrCol extends VectorExpression {
     return "boolean";
   }
 
+  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;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
index a90dc30..bb37a04 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorizationContext.java
@@ -622,6 +622,8 @@ public class TestVectorizationContext {
     assertEquals(veAnd.getClass(), ColAndCol.class);
     assertEquals(1, veAnd.getChildExpressions().length);
     assertEquals(veAnd.getChildExpressions()[0].getClass(), LongColGreaterLongScalar.class);
+    assertEquals(2, ((ColAndCol) veAnd).getColNum1());
+    assertEquals(1, ((ColAndCol) veAnd).getColNum2());
     assertEquals(3, ((ColAndCol) veAnd).getOutputColumn());
 
     //OR
@@ -645,6 +647,8 @@ public class TestVectorizationContext {
     assertEquals(veOr.getClass(), ColOrCol.class);
     assertEquals(1, veAnd.getChildExpressions().length);
     assertEquals(veAnd.getChildExpressions()[0].getClass(), LongColGreaterLongScalar.class);
+    assertEquals(2, ((ColOrCol) veOr).getColNum1());
+    assertEquals(1, ((ColOrCol) veOr).getColNum2());
     assertEquals(3, ((ColOrCol) veOr).getOutputColumn());
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
index 16c4198..7d54a9c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorLogicalExpressions.java
@@ -40,7 +40,7 @@ public class TestVectorLogicalExpressions {
   @Test
   public void testLongColOrLongCol() {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
-    ColOrCol expr = new ColOrCol(new int[] {0, 1}, 2);
+    ColOrCol expr = new ColOrCol(0, 1, 2);
     LongColumnVector outCol = (LongColumnVector) batch.cols[2];
     expr.evaluate(batch);
     // verify
@@ -96,7 +96,7 @@ public class TestVectorLogicalExpressions {
   @Test
   public void testLongColAndLongCol() {
     VectorizedRowBatch batch = getBatchThreeBooleanCols();
-    ColAndCol expr = new ColAndCol(new int[] {0, 1}, 2);
+    ColAndCol expr = new ColAndCol(0, 1, 2);
     LongColumnVector outCol = (LongColumnVector) batch.cols[2];
     expr.evaluate(batch);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/5c58dcee/ql/src/test/queries/clientpositive/vector_multi_and_projection.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/vector_multi_and_projection.q b/ql/src/test/queries/clientpositive/vector_multi_and_projection.q
deleted file mode 100644
index 21ad36e..0000000
--- a/ql/src/test/queries/clientpositive/vector_multi_and_projection.q
+++ /dev/null
@@ -1,196 +0,0 @@
-set hive.cli.print.header=true;
-set hive.explain.user=false;
-SET hive.auto.convert.join=true;
-set hive.fetch.task.conversion=none;
-set hive.mapred.mode=nonstrict;
-
--- SORT_QUERY_RESULTS
-
-create table vectortab2k(
-            t tinyint,
-            si smallint,
-            i int,
-            b bigint,
-            f float,
-            d double,
-            dc decimal(38,18),
-            bo boolean,
-            s string,
-            s2 string,
-            ts timestamp,
-            ts2 timestamp,
-            dt date)
-ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
-STORED AS TEXTFILE;
-
-LOAD DATA LOCAL INPATH '../../data/files/vectortab2k' OVERWRITE INTO TABLE vectortab2k;
-
-CREATE TABLE scratch AS SELECT t, si, i, b, bo FROM vectortab2k;
-INSERT INTO TABLE scratch VALUES (NULL, NULL, NULL, NULL, NULL);
-
-CREATE TABLE vectortab2k_orc STORED AS ORC AS SELECT * FROM scratch;
-
-SET hive.vectorized.execution.enabled=true;
-
-EXPLAIN
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
-        order by t, si, i) as q;
-
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
-        order by t, si, i) as q;
-
-EXPLAIN
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
-        order by t, si, i, b) as q;
-
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
-        order by t, si, i, b) as q;
-
--- Use a boolean column rather than a column comparison expression.
-EXPLAIN
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
-        order by t, si, bo, b) as q;
-
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
-        order by t, si, bo, b) as q;
-
--- With some filtering
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, i, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc
-        where pmod(t, 4) > 1
-        order by t, si, i) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, i, b, (t < 0) as child1, (si > 0) as child2, (i < 0) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND i < 0 AND b > 0) as multi_and_col from vectortab2k_orc
-        where pmod(t, 4) < 2
-        order by t, si, i, b) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, bo, b, (t < 0) as child1, (si > 0) as child2, (bo) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo AND b > 0) as multi_and_col from vectortab2k_orc
-        where pmod(b, 8) == 7
-        order by t, si, bo, b) as q;
-
-SET hive.vectorized.execution.enabled=false;
-
-CREATE TABLE scratch_repeat AS SELECT t, si, i, b, bo, 20 as t_repeat,
-     9000 as si_repeat, 9233320 as i_repeat, -823823999339992 as b_repeat, false as bo_repeat_false, true as bo_repeat_true FROM vectortab2k;
-
--- The repeated columns ought to create repeated VectorizedRowBatch for those columns.
--- And then when we do a comparison, we should generate a repeated boolean result.
-CREATE TABLE vectortab2k_orc_repeat STORED AS ORC AS SELECT * FROM scratch_repeat;
-
-SET hive.vectorized.execution.enabled=true;
-
--- t_repeat > 0 should generate all true.
-SELECT sum(hash(*)) FROM
-    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
-        order by t_repeat, si, i) as q;
-
--- t_repeat < 0 should generate all false.
-SELECT sum(hash(*)) FROM
-    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
-        order by t_repeat, si, i) as q;
-
--- Two repeated false columns at beginning...
-SELECT sum(hash(*)) FROM
-    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
-        order by t_repeat, si_repeat, i) as q;
-
--- si_repeat > 0 should generate all true.
-SELECT sum(hash(*)) FROM
-    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
-        order by t, si_repeat, i, b_repeat) as q;
-
--- si_repeat < 0 should generate all false.
-SELECT sum(hash(*)) FROM
-    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
-        order by t, si_repeat, i, b_repeat) as q;
-
--- Use a boolean column rather than a column comparison expression.
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
-        order by t, si, bo_repeat_false, b) as q;
-
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
-        order by t, si, bo_repeat_true, b) as q;
-
--- With some filtering
-SELECT sum(hash(*)) FROM
-    (SELECT t_repeat, si, i, (t_repeat > 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
-        where pmod(si, 4) = 0
-        order by t_repeat, si, i) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t_repeat, si, i, (t_repeat < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_repeat < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
-        where pmod(si, 4) = 3
-        order by t_repeat, si, i) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t_repeat, si_repeat, i, (t_repeat > 0) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (t_repeat > 0 AND si_repeat > 0 AND i < 0) as multi_and_col from vectortab2k_orc_repeat
-        where pmod(si, 4) != 3
-        order by t_repeat, si_repeat, i) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t, si_repeat, i, b_repeat, (t < 0 ) as child1, (si_repeat > 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat > 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
-        where pmod(si, 4) < 2
-        order by t, si_repeat, i, b_repeat) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t, si_repeat, i, b_repeat, (t < 0) as child1, (si_repeat < 0) as child2, (i < 0) as child3, (b_repeat > 0) as child4, (t < 0 AND si_repeat < 0 AND i < 0 AND b_repeat > 0) as multi_and_col from vectortab2k_orc_repeat
-        where pmod(t, 4) = 0
-        order by t, si_repeat, i, b_repeat) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, bo_repeat_false, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_false) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_false AND b > 0) as multi_and_col from vectortab2k_orc_repeat
-        where pmod(b, 4) > 1
-        order by t, si, bo_repeat_false, b) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t, si, bo_repeat_true, b, (t < 0) as child1, (si > 0) as child2, (bo_repeat_true) as child3, (b > 0) as child4, (t < 0 AND si > 0 AND bo_repeat_true AND b > 0) as multi_and_col from vectortab2k_orc_repeat
-        where pmod(si, 4) < 3
-        order by t, si, bo_repeat_true, b) as q;
-
-
-SET hive.vectorized.execution.enabled=false;
-
-CREATE TABLE scratch_null AS SELECT t, si, i, b, bo,
-     cast(null as tinyint) as t_null, cast(null as smallint) as si_null, cast(null as int) as i_null, cast(null as bigint) as b_null, cast(null as boolean) as bo_null FROM vectortab2k;
-
--- The nulled columns ought to create repeated null VectorizedRowBatch for those columns.
-CREATE TABLE vectortab2k_orc_null STORED AS ORC AS SELECT * FROM scratch_null;
-
-SET hive.vectorized.execution.enabled=true;
-
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
-        order by t_null, si, i) as q;
-
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
-        order by t_null, si, i) as q;
-
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
-        order by t_null, si_null, i) as q;
-    
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
-        order by t_null, si_null, i_null) as q;
-
--- With some filtering
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si, i, (t_null is null) as child1, (si > 0) as child2, (i < 0) as child3, (t_null is null AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
-        where pmod(i,4) = 3
-        order by t_null, si, i) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si, i, (t_null < 0) as child1, (si > 0) as child2, (i < 0) as child3, (t_null < 0 AND si > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
-        where pmod(i,4) = 2
-        order by t_null, si, i) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si_null, i, (t_null is null) as child1, (si_null > 0) as child2, (i < 0) as child3, (t_null is null AND si_null > 0 AND i < 0) as multi_and_col from vectortab2k_orc_null
-        where pmod(i,4) != 3
-        order by t_null, si_null, i) as q;
-SELECT sum(hash(*)) FROM
-    (SELECT t_null, si_null, i_null, (t_null is null) as child1, (si_null is null) as child2, (i_null is null) as child3, (t_null is null AND si_null is null AND i_null is null) as multi_and_col from vectortab2k_orc_null
-        where pmod(i,4) < 3
-        order by t_null, si_null, i_null) as q;
-