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/04/23 23:41:06 UTC

svn commit: r1471159 [1/4] - in /hive/branches/vectorization/ql/src: java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/ java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/ test/org/apache/hadoop/hive/ql/exec/vector/expressions/

Author: hashutosh
Date: Tue Apr 23 21:41:05 2013
New Revision: 1471159

URL: http://svn.apache.org/r1471159
Log:
HIVE-4379 : Implement Vectorized Column-Column expressions (Jitendra Nath Pandey via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColNotEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColNotEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColNotEqualDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColNotEqualLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColAddDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColAddLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColDivideDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColDivideLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColMultiplyDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColMultiplyLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColSubtractDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColSubtractLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/ColumnArithmeticColumn.txt
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterColumnCompareColumn.txt
Modified:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorArithmeticExpressions.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorFilterExpressions.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorizedRowGroupGenUtil.java

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColAddDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColAddDoubleColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] + vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] + vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] + vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] + vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] + vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColAddLongColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColAddLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] + vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] + vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] + vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] + vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] + vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColDivideDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColDivideDoubleColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] / vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] / vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] / vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] / vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] / vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColDivideLongColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColDivideLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] / vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] / vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] / vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] / vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] / vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColMultiplyDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColMultiplyDoubleColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] * vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] * vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] * vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] * vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] * vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColMultiplyLongColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColMultiplyLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] * vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] * vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] * vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] * vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] * vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColSubtractDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColSubtractDoubleColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] - vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] - vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] - vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] - vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] - vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,173 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColSubtractLongColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColSubtractLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] - vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] - vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] - vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] - vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] - vector2[0];
+        }
+      }
+    } else {
+      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];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,222 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class FilterDoubleColEqualDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+
+  public FilterDoubleColEqualDoubleColumn(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(vector1[0] == vector2[0])) {
+          batch.size = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[0] == vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[0] == vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[i] == vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[i] == vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] == vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] ==  vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      if (nullPos1[0] || nullPos2[0]) {
+        batch.size = 0; 
+      } 
+    } else if (inputColVector1.isRepeating) {
+      if (nullPos1[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (vector1[0] == vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (vector1[0] == vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (nullPos2[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (vector1[i] == vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (vector1[i] == vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (batch.selectedInUse) {
+      int newSize = 0;
+      for(int j=0; j != n; j++) {
+        int i = sel[j];
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] == vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      batch.size = newSize;
+    } else {
+      int newSize = 0;
+      for(int i = 0; i != n; i++) {
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] == vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      if (newSize < batch.size) {
+        batch.size = newSize;
+        batch.selectedInUse = true;
+      }
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,222 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class FilterDoubleColEqualLongColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+
+  public FilterDoubleColEqualLongColumn(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(vector1[0] == vector2[0])) {
+          batch.size = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[0] == vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[0] == vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[i] == vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[i] == vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] == vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] ==  vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      if (nullPos1[0] || nullPos2[0]) {
+        batch.size = 0; 
+      } 
+    } else if (inputColVector1.isRepeating) {
+      if (nullPos1[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (vector1[0] == vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (vector1[0] == vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (nullPos2[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (vector1[i] == vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (vector1[i] == vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (batch.selectedInUse) {
+      int newSize = 0;
+      for(int j=0; j != n; j++) {
+        int i = sel[j];
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] == vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      batch.size = newSize;
+    } else {
+      int newSize = 0;
+      for(int i = 0; i != n; i++) {
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] == vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      if (newSize < batch.size) {
+        batch.size = newSize;
+        batch.selectedInUse = true;
+      }
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,222 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class FilterDoubleColGreaterDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+
+  public FilterDoubleColGreaterDoubleColumn(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(vector1[0] > vector2[0])) {
+          batch.size = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[0] > vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[0] > vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[i] > vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[i] > vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] > vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] >  vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      if (nullPos1[0] || nullPos2[0]) {
+        batch.size = 0; 
+      } 
+    } else if (inputColVector1.isRepeating) {
+      if (nullPos1[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (vector1[0] > vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (vector1[0] > vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (nullPos2[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (vector1[i] > vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (vector1[i] > vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (batch.selectedInUse) {
+      int newSize = 0;
+      for(int j=0; j != n; j++) {
+        int i = sel[j];
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] > vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      batch.size = newSize;
+    } else {
+      int newSize = 0;
+      for(int i = 0; i != n; i++) {
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] > vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      if (newSize < batch.size) {
+        batch.size = newSize;
+        batch.selectedInUse = true;
+      }
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualDoubleColumn.java?rev=1471159&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualDoubleColumn.java Tue Apr 23 21:41:05 2013
@@ -0,0 +1,222 @@
+/**
+ * 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.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class FilterDoubleColGreaterEqualDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+
+  public FilterDoubleColGreaterEqualDoubleColumn(int colNum1, int colNum2) { 
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    int[] sel = batch.selected;
+    boolean[] nullPos1 = inputColVector1.isNull;
+    boolean[] nullPos2 = inputColVector2.isNull;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector1.noNulls && inputColVector2.noNulls) {
+      if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(vector1[0] >= vector2[0])) {
+          batch.size = 0;
+        }
+      } else if (inputColVector1.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[0] >= vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[0] >= vector2[i]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (inputColVector2.isRepeating) {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (vector1[i] >= vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (vector1[i] >= vector2[0]) {
+              sel[newSize++] = i;
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (vector1[i] >= vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector1[i] >=  vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < batch.size) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else if (inputColVector1.isRepeating && inputColVector2.isRepeating) {
+      if (nullPos1[0] || nullPos2[0]) {
+        batch.size = 0; 
+      } 
+    } else if (inputColVector1.isRepeating) {
+      if (nullPos1[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos2[i]) {
+              if (vector1[0] >= vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos2[i]) {
+              if (vector1[0] >= vector2[i]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (nullPos2[0]) {
+        batch.size = 0;
+      } else {
+        if (batch.selectedInUse) {
+          int newSize = 0;
+          for(int j=0; j != n; j++) {
+            int i = sel[j];
+            if (!nullPos1[i]) {
+              if (vector1[i] >= vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          batch.size = newSize;
+        } else {
+          int newSize = 0;
+          for(int i = 0; i != n; i++) {
+            if (!nullPos1[i]) {
+              if (vector1[i] >= vector2[0]) {
+                sel[newSize++] = i;
+              }
+            }
+          }
+          if (newSize < batch.size) {
+            batch.size = newSize;
+            batch.selectedInUse = true;
+          }
+        }
+      }
+    } else if (batch.selectedInUse) {
+      int newSize = 0;
+      for(int j=0; j != n; j++) {
+        int i = sel[j];
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] >= vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      batch.size = newSize;
+    } else {
+      int newSize = 0;
+      for(int i = 0; i != n; i++) {
+        if (!nullPos1[i] && !nullPos2[i]) {
+          if (vector1[i] >= vector2[i]) {
+            sel[newSize++] = i;
+          }
+        }
+      }
+      if (newSize < batch.size) {
+        batch.size = newSize;
+        batch.selectedInUse = true;
+      }
+    }
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+}