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 09:09:57 UTC

svn commit: r1470830 [1/2] - 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 07:09:56 2013
New Revision: 1470830

URL: http://svn.apache.org/r1470830
Log:
HIVE-4380 : Implement Vectorized Scalar-Column expressions (Eric Hanson via Jitendra Pandey)

Added:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarSubtractDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarSubtractLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/ScalarArithmeticColumn.txt
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java
Modified:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarAddDoubleColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarAddDoubleColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value + vector[0];
+      
+      // 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] = value + vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value + vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+        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/DoubleScalarAddLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarAddLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarAddLongColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarAddLongColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) 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;
+    long[] 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] = value + vector[0];
+      
+      // 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] = value + vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value + vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+        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/DoubleScalarDivideDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarDivideDoubleColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarDivideDoubleColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value / vector[0];
+      
+      // 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] = value / vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value / vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+        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/DoubleScalarDivideLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarDivideLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarDivideLongColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarDivideLongColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) 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;
+    long[] 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] = value / vector[0];
+      
+      // 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] = value / vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value / vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+        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/DoubleScalarModuloDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarModuloDoubleColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarModuloDoubleColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value % vector[0];
+      
+      // 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] = value % vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value % vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+        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/DoubleScalarModuloLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarModuloLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarModuloLongColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarModuloLongColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) 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;
+    long[] 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] = value % vector[0];
+      
+      // 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] = value % vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value % vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+        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/DoubleScalarMultiplyDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarMultiplyDoubleColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarMultiplyDoubleColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value * vector[0];
+      
+      // 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] = value * vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value * vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+        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/DoubleScalarMultiplyLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarMultiplyLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarMultiplyLongColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarMultiplyLongColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) 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;
+    long[] 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] = value * vector[0];
+      
+      // 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] = value * vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value * vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+        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/DoubleScalarSubtractDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarSubtractDoubleColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarSubtractDoubleColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value - vector[0];
+      
+      // 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] = value - vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value - vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value - vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value - vector[i];
+        }
+        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/DoubleScalarSubtractLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleScalarSubtractLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class DoubleScalarSubtractLongColumn extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleScalarSubtractLongColumn(double value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) 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;
+    long[] 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] = value - vector[0];
+      
+      // 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] = value - vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value - vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value - vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value - vector[i];
+        }
+        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/LongScalarAddDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarAddDoubleColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarAddDoubleColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value + vector[0];
+      
+      // 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] = value + vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value + vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+        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/LongScalarAddLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarAddLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarAddLongColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarAddLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] 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] = value + vector[0];
+      
+      // 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] = value + vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value + vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value + vector[i];
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/LongScalarDivideDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarDivideDoubleColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarDivideDoubleColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value / vector[0];
+      
+      // 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] = value / vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value / vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+        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/LongScalarDivideLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarDivideLongColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarDivideLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] 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] = value / vector[0];
+      
+      // 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] = value / vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value / vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value / vector[i];
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/LongScalarModuloDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarModuloDoubleColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarModuloDoubleColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value % vector[0];
+      
+      // 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] = value % vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value % vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+        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/LongScalarModuloLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarModuloLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarModuloLongColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarModuloLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] 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] = value % vector[0];
+      
+      // 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] = value % vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value % vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value % vector[i];
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @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/LongScalarMultiplyDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyDoubleColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyDoubleColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarMultiplyDoubleColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarMultiplyDoubleColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch 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] = value * vector[0];
+      
+      // 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] = value * vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value * vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+        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/LongScalarMultiplyLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyLongColumn.java?rev=1470830&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarMultiplyLongColumn.java Tue Apr 23 07:09:56 2013
@@ -0,0 +1,118 @@
+/**
+ * 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;
+
+/*
+ * Because of the templatized nature of the code, either or both
+ * of these ColumnVector imports may be needed. Listing both of them
+ * rather than using ....vectorization.*;
+ */
+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;
+
+/**
+ * Implements a vectorized arithmetic operator with a scalar on the left and a
+ * column vector on the right. The result is output to an output column vector.
+ */
+public class LongScalarMultiplyLongColumn extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public LongScalarMultiplyLongColumn(long value, int colNum, int outputColumn) {
+    this.colNum = colNum;
+    this.value = value;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  /**
+   * Method to evaluate scalar-column operation in vectorized fashion.
+   *
+   * @batch a package of rows with each column stored in a vector
+   */
+  public void evaluate(VectorizedRowBatch batch) {
+    LongColumnVector inputColVector = (LongColumnVector) batch.cols[colNum];
+    LongColumnVector outputColVector = (LongColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    boolean[] inputIsNull = inputColVector.isNull;
+    boolean[] outputIsNull = outputColVector.isNull;
+    outputColVector.noNulls = inputColVector.noNulls;
+    int n = batch.size;
+    long[] vector = inputColVector.vector;
+    long[] 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] = value * vector[0];
+      
+      // 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] = value * vector[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+      }
+      outputColVector.isRepeating = false;
+    } else {                         /* there are nulls */ 
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = value * vector[i];
+          outputIsNull[i] = inputIsNull[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = value * vector[i];
+        }
+        System.arraycopy(inputIsNull, 0, outputIsNull, 0, n);
+      }
+      outputColVector.isRepeating = false;
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+  
+  @Override
+  public String getOutputType() {
+    return "long";
+  }
+}