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/21 00:33:28 UTC

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

Author: hashutosh
Date: Sat Apr 20 22:33:26 2013
New Revision: 1470247

URL: http://svn.apache.org/r1470247
Log:
HIVE-4282 : Implement vectorized column-scalar expressions (Jitendra Nath Pandey via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColNotEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColLessLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColNotEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColNotEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColGreaterLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColLessLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColNotEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterLongColNotEqualLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColAddDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColAddLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColDivideDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColDivideLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColGreaterDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColGreaterEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColLessDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColLessEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColMultiplyDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColMultiplyLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColNotEqualDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColSubtractDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColSubtractLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/ColumnArithmeticScalar.txt
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/ColumnCompareScalar.txt
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/FilterColumnCompareScalar.txt
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/
    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/VectorExpression.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpression.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Base class for expressions.
+ */
+public abstract class VectorExpression {
+
+  /**
+   * Child expressions are evaluated post order.
+   */
+  protected VectorExpression [] childExpressions = null;
+
+  /**
+   * This is the primary method to implement expression logic.
+   * @param vrg
+   */
+  public abstract void evaluate(VectorizedRowBatch vrg);
+
+  /**
+   * Returns the index of the output column in the array
+   * of column vectors. If not applicable, -1 is returned.
+   * @return Index of the output column
+   */
+  public abstract int getOutputColumn();
+
+  /**
+   * Returns type of the output column.
+   */
+  public abstract String getOutputType();
+
+  /**
+   * Initialize the child expressions.
+   */
+  public void setChildExpressions(VectorExpression [] ve) {
+    childExpressions = ve;
+  }
+
+  public VectorExpression[] getChildExpressions() {
+    return childExpressions;
+  }
+
+  /**
+   * Evaluate the child expressions on the given input batch.
+   * @param vrg {@link VectorizedRowBatch}
+   */
+  final protected void evaluateChildren(VectorizedRowBatch vrg) {
+    if (childExpressions != null) {
+      for (VectorExpression ve : childExpressions) {
+        ve.evaluate(vrg);
+      }
+    }
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColAddDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColAddDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] + value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] + value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] + value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] + value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] + value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/DoubleColAddLongScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColAddLongScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColAddLongScalar extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public DoubleColAddLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] + value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] + value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] + value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] + value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] + value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/DoubleColDivideDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColDivideDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColDivideDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] / value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] / value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] / value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] / value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] / value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/DoubleColDivideLongScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColDivideLongScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColDivideLongScalar extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public DoubleColDivideLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] / value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] / value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] / value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] / value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] / value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/DoubleColEqualDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColEqualDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColEqualDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColEqualDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,114 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColEqualDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColEqualDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] == value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] == value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] == value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] == value ? 1 : 0;
+        } else {
+          outputVector[0] = 0;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] == value ? 1 : 0;
+          } else {
+            //compare with null is false
+            outputVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] == value ? 1 : 0;
+          } else {
+            outputVector[i] = 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,114 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColGreaterDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColGreaterDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] > value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] > value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] > value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] > value ? 1 : 0;
+        } else {
+          outputVector[0] = 0;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] > value ? 1 : 0;
+          } else {
+            //compare with null is false
+            outputVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] > value ? 1 : 0;
+          } else {
+            outputVector[i] = 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterEqualDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterEqualDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterEqualDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColGreaterEqualDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,114 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColGreaterEqualDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColGreaterEqualDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] >= value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] >= value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] >= value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] >= value ? 1 : 0;
+        } else {
+          outputVector[0] = 0;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] >= value ? 1 : 0;
+          } else {
+            //compare with null is false
+            outputVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] >= value ? 1 : 0;
+          } else {
+            outputVector[i] = 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,114 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColLessDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColLessDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] < value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] < value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] < value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] < value ? 1 : 0;
+        } else {
+          outputVector[0] = 0;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] < value ? 1 : 0;
+          } else {
+            //compare with null is false
+            outputVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] < value ? 1 : 0;
+          } else {
+            outputVector[i] = 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessEqualDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessEqualDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessEqualDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColLessEqualDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,114 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColLessEqualDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColLessEqualDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] <= value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] <= value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] <= value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] <= value ? 1 : 0;
+        } else {
+          outputVector[0] = 0;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] <= value ? 1 : 0;
+          } else {
+            //compare with null is false
+            outputVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] <= value ? 1 : 0;
+          } else {
+            outputVector[i] = 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColMultiplyDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColMultiplyDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] * value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] * value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] * value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] * value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] * value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/DoubleColMultiplyLongScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColMultiplyLongScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColMultiplyLongScalar extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public DoubleColMultiplyLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] * value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] * value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] * value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] * value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] * value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/DoubleColNotEqualDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColNotEqualDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColNotEqualDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColNotEqualDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,114 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColNotEqualDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColNotEqualDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    long[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        outputVector[0] = vector[0] != value ? 1 : 0;
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] != value ? 1 : 0;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] != value ? 1 : 0;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          outputVector[0] = vector[0] != value ? 1 : 0;
+        } else {
+          outputVector[0] = 0;
+        }
+        outputColVector.isRepeating = true;
+      } else if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] != value ? 1 : 0;
+          } else {
+            //compare with null is false
+            outputVector[i] = 0;
+          }
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            outputVector[i] = vector[i] != value ? 1 : 0;
+          } else {
+            outputVector[i] = 0;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColSubtractDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColSubtractDoubleScalar(int colNum, double value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] - value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] - value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] - value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] - value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] - value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/DoubleColSubtractLongScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColSubtractLongScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,104 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColSubtractLongScalar extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public DoubleColSubtractLongScalar(int colNum, long value, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+
+    if (inputColVector.isRepeating) {
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector[0] - value;
+      // Even if there are no nulls, we always copy over entry 0. Simplifies code.
+      outputIsNull[0] = inputIsNull[0]; 
+      outputColVector.isRepeating = true;
+    } else if (inputColVector.noNulls) {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] - value;
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] - value;
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else /* there are nulls */ {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector[i] - value;
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector[i] - value;
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/FilterDoubleColEqualDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,132 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+
+public class FilterDoubleColEqualDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+
+  public FilterDoubleColEqualDoubleScalar(int colNum, double value) { 
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(vector[0] == value)) {
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (vector[i] == value) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector[i] == value) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(vector[0] == value)) {
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (vector[i] == value) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (vector[i] == value) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColEqualLongScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,132 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+
+public class FilterDoubleColEqualLongScalar extends VectorExpression {
+  private int colNum;
+  private long value;
+
+  public FilterDoubleColEqualLongScalar(int colNum, long value) { 
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(vector[0] == value)) {
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (vector[i] == value) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector[i] == value) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(vector[0] == value)) {
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (vector[i] == value) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (vector[i] == value) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleScalar.java?rev=1470247&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/FilterDoubleColGreaterDoubleScalar.java Sat Apr 20 22:33:26 2013
@@ -0,0 +1,132 @@
+/**
+ * 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.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+
+public class FilterDoubleColGreaterDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+
+  public FilterDoubleColGreaterDoubleScalar(int colNum, double value) { 
+    this.colNum = colNum;
+    this.value = value;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector = (DoubleColumnVector) batch.cols[colNum];
+    int[] sel = batch.selected;
+    boolean[] nullPos = inputColVector.isNull;
+    int n = batch.size;
+    double[] vector = inputColVector.vector;
+
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    if (inputColVector.noNulls) {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!(vector[0] > value)) {
+          //Entire batch is filtered out.
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (vector[i] > value) {
+            sel[newSize++] = i;
+          }
+        }
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (vector[i] > value) {
+            sel[newSize++] = i;
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    } else {
+      if (inputColVector.isRepeating) {
+        //All must be selected otherwise size would be zero
+        //Repeating property will not change.
+        if (!nullPos[0]) {
+          if (!(vector[0] > value)) {
+            //Entire batch is filtered out.
+            batch.size = 0;
+          }
+        } else {
+          batch.size = 0;
+        }
+      } else if (batch.selectedInUse) {
+        int newSize = 0;
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          if (!nullPos[i]) {
+           if (vector[i] > value) {
+             sel[newSize++] = i;
+           }
+          }
+        }
+        //Change the selected vector
+        batch.size = newSize;
+      } else {
+        int newSize = 0;
+        for(int i = 0; i != n; i++) {
+          if (!nullPos[i]) {
+            if (vector[i] > value) {
+              sel[newSize++] = i;
+            }
+          }
+        }
+        if (newSize < n) {
+          batch.size = newSize;
+          batch.selectedInUse = true;
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return -1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "boolean";
+  }
+}