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/05/08 06:23:50 UTC

svn commit: r1480156 - 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: Wed May  8 04:23:50 2013
New Revision: 1480156

URL: http://svn.apache.org/r1480156
Log:
HIVE-4462 : Finish support for modulo (%) operator for vectorized arithmetic (Eric Hanson via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleScalar.java
Modified:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleColumn.java?rev=1480156&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleColumn.java Wed May  8 04:23:50 2013
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColModuloDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColModuloDoubleColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    /* Set repeating property to false (the default).
+     * It will be set to true later if needed later.
+     */
+    outputColVector.isRepeating = false;
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] % vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] % vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] % vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] % vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] % vector2[0];
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] % vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] %  vector2[i];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleScalar.java?rev=1480156&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloDoubleScalar.java Wed May  8 04:23:50 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 DoubleColModuloDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public DoubleColModuloDoubleScalar(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/DoubleColModuloLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongColumn.java?rev=1480156&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongColumn.java Wed May  8 04:23:50 2013
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class DoubleColModuloLongColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public DoubleColModuloLongColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    DoubleColumnVector inputColVector1 = (DoubleColumnVector) batch.cols[colNum1];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    double[] vector1 = inputColVector1.vector;
+    long[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    /* Set repeating property to false (the default).
+     * It will be set to true later if needed later.
+     */
+    outputColVector.isRepeating = false;
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] % vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] % vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] % vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] % vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] % vector2[0];
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] % vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] %  vector2[i];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongScalar.java?rev=1480156&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/DoubleColModuloLongScalar.java Wed May  8 04:23:50 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 DoubleColModuloLongScalar extends VectorExpression {
+  private int colNum;
+  private long value;
+  private int outputColumn;
+
+  public DoubleColModuloLongScalar(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/LongColModuloDoubleColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleColumn.java?rev=1480156&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleColumn.java Wed May  8 04:23:50 2013
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+package org.apache.hadoop.hive.ql.exec.vector.expressions.gen;
+
+import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.*;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class LongColModuloDoubleColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public LongColModuloDoubleColumn(int colNum1, int colNum2, int outputColumn) {
+    this.colNum1 = colNum1;
+    this.colNum2 = colNum2;
+    this.outputColumn = outputColumn;
+  }
+
+  @Override
+  public void evaluate(VectorizedRowBatch batch) {
+
+    if (childExpressions != null) {
+      super.evaluateChildren(batch);
+    }
+
+    LongColumnVector inputColVector1 = (LongColumnVector) batch.cols[colNum1];
+    DoubleColumnVector inputColVector2 = (DoubleColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    long[] vector1 = inputColVector1.vector;
+    double[] vector2 = inputColVector2.vector;
+
+    double[] outputVector = outputColVector.vector;
+    
+    // return immediately if batch is empty
+    if (n == 0) {
+      return;
+    }
+    
+    /* Set repeating property to false (the default).
+     * It will be set to true later if needed later.
+     */
+    outputColVector.isRepeating = false;
+
+    //Handle nulls first
+    if (inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isNull[0] = true;
+        outputColVector.isRepeating = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector2.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i];
+          }
+        }
+      }
+    } else if (!inputColVector1.noNulls && !inputColVector2.noNulls) {
+      outputColVector.noNulls = false;
+      if (inputColVector1.isRepeating || inputColVector2.isRepeating) {
+        //Output will also be repeating and null
+        outputColVector.isRepeating = true;
+        outputColVector.isNull[0] = true;
+        //return as no further processing is needed
+        return;
+      } else {
+        if (batch.selectedInUse) {
+          for(int j = 0; j != n; j++) {
+            int i = sel[j];
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        } else {
+          for(int i = 0; i != n; i++) {
+            outputColVector.isNull[i] = inputColVector1.isNull[i] || inputColVector2.isNull[i];
+          }
+        }
+      }
+    }
+
+
+    //Disregard nulls for processing
+    if (inputColVector1.isRepeating && inputColVector2.isRepeating) { 
+      //All must be selected otherwise size would be zero
+      //Repeating property will not change.
+      outputVector[0] = vector1[0] % vector2[0];
+      outputColVector.isRepeating = true;
+    } else if (inputColVector1.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[0] % vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] % vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] % vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] % vector2[0];
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] % vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] %  vector2[i];
+        }
+      }
+    }
+  }
+
+  @Override
+  public int getOutputColumn() {
+    return outputColumn;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "double";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleScalar.java?rev=1480156&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColModuloDoubleScalar.java Wed May  8 04:23:50 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.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class LongColModuloDoubleScalar extends VectorExpression {
+  private int colNum;
+  private double value;
+  private int outputColumn;
+
+  public LongColModuloDoubleScalar(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);
+    }
+
+    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] = 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";
+  }
+}

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java?rev=1480156&r1=1480155&r2=1480156&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java Wed May  8 04:23:50 2013
@@ -21,17 +21,19 @@ public class CodeGen {
       {"ColumnArithmeticScalar", "Subtract", "long", "double", "-"},
       {"ColumnArithmeticScalar", "Multiply", "long", "double", "*"},
       {"ColumnArithmeticScalar", "Divide", "long", "double", "/"},
-      // TODO add Modulo for all the long/double, double/double, and double/long combinations
+      {"ColumnArithmeticScalar", "Modulo", "long", "double", "%"},
 
       {"ColumnArithmeticScalar", "Add", "double", "long", "+"},
       {"ColumnArithmeticScalar", "Subtract", "double", "long", "-"},
       {"ColumnArithmeticScalar", "Multiply", "double", "long", "*"},
       {"ColumnArithmeticScalar", "Divide", "double", "long", "/"},
+      {"ColumnArithmeticScalar", "Modulo", "double", "long", "%"},
 
       {"ColumnArithmeticScalar", "Add", "double", "double", "+"},
       {"ColumnArithmeticScalar", "Subtract", "double", "double", "-"},
       {"ColumnArithmeticScalar", "Multiply", "double", "double", "*"},
       {"ColumnArithmeticScalar", "Divide", "double", "double", "/"},
+      {"ColumnArithmeticScalar", "Modulo", "double", "double", "%"},
 
       {"ScalarArithmeticColumn", "Add", "long", "long", "+"},
       {"ScalarArithmeticColumn", "Subtract", "long", "long", "-"},
@@ -67,16 +69,19 @@ public class CodeGen {
       {"ColumnArithmeticColumn", "Subtract", "long", "double", "-"},
       {"ColumnArithmeticColumn", "Multiply", "long", "double", "*"},
       {"ColumnArithmeticColumn", "Divide", "long", "double", "/"},
+      {"ColumnArithmeticColumn", "Modulo", "long", "double", "%"},
 
       {"ColumnArithmeticColumn", "Add", "double", "long", "+"},
       {"ColumnArithmeticColumn", "Subtract", "double", "long", "-"},
       {"ColumnArithmeticColumn", "Multiply", "double", "long", "*"},
       {"ColumnArithmeticColumn", "Divide", "double", "long", "/"},
+      {"ColumnArithmeticColumn", "Modulo", "double", "long", "%"},
 
       {"ColumnArithmeticColumn", "Add", "double", "double", "+"},
       {"ColumnArithmeticColumn", "Subtract", "double", "double", "-"},
       {"ColumnArithmeticColumn", "Multiply", "double", "double", "*"},
       {"ColumnArithmeticColumn", "Divide", "double", "double", "/"},
+      {"ColumnArithmeticColumn", "Modulo", "double", "double", "%"},
 
       {"ColumnCompareScalar", "Equal", "long", "double", "=="},
       {"ColumnCompareScalar", "Equal", "double", "double", "=="},

Modified: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java?rev=1480156&r1=1480155&r2=1480156&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java (original)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorScalarColArithmetic.java Wed May  8 04:23:50 2013
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec.v
 import junit.framework.Assert;
 
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongScalarSubtractLongColumn;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.LongScalarModuloLongColumn;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TestVectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -43,6 +44,33 @@ public class TestVectorScalarColArithmet
     return batch;
   }
 
+  private VectorizedRowBatch getBatchSingleLongVectorPositiveNonZero() {
+    VectorizedRowBatch batch = new VectorizedRowBatch(2);
+    final int size = VectorizedRowBatch.DEFAULT_SIZE;
+    LongColumnVector lcv = new LongColumnVector();
+    for (int i = 0; i < size; i++) {
+      lcv.vector[i] = (i + 1) * 37;
+    }
+    batch.cols[0] = lcv;
+    batch.cols[1] = new LongColumnVector();
+    batch.size = size;
+    return batch;
+  }
+  
+  @Test
+  public void testLongScalarModuloLongColNoNulls()  {
+    VectorizedRowBatch batch = getBatchSingleLongVectorPositiveNonZero();
+    LongScalarModuloLongColumn expr = new LongScalarModuloLongColumn(100, 0, 1);
+    expr.evaluate(batch);
+    
+    // verify
+    for (int i = 0; i < VectorizedRowBatch.DEFAULT_SIZE; i++) {
+      Assert.assertEquals(100 % ((i + 1) * 37), ((LongColumnVector) batch.cols[1]).vector[i]);
+    }
+    Assert.assertTrue(((LongColumnVector)batch.cols[1]).noNulls);
+    Assert.assertFalse(((LongColumnVector)batch.cols[1]).isRepeating);
+  }
+  
   @Test
   public void testLongScalarSubtractLongColNoNulls()  {
     VectorizedRowBatch batch = getVectorizedRowBatchSingleLongVector(