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/12 20:53:16 UTC

svn commit: r1481622 - in /hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec: ./ vector/ vector/expressions/ vector/expressions/gen/ vector/expressions/templates/

Author: hashutosh
Date: Sun May 12 18:53:15 2013
New Revision: 1481622

URL: http://svn.apache.org/r1481622
Log:
HIVE-4509 : Integer division should be cast to double. (Jitendra Nath Pandey via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java
Removed:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColDivideLongColumn.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongColDivideLongScalar.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/gen/LongScalarDivideLongColumn.java
Modified:
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
    hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/templates/CodeGen.java

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java?rev=1481622&r1=1481621&r2=1481622&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java Sun May 12 18:53:15 2013
@@ -573,20 +573,17 @@ public class ExecDriver extends Task<Map
           validateVectorOperator(vectorOp);
         } catch (HiveException e) {
           LOG.info("Cannot vectorize the plan", e);
-          System.out.println("Cannot vectorize the plan");
+          System.out.println("Cannot vectorize the plan: "+e.getMessage());
           return false;
         }
       }
     }
-    System.out.println("Query can be vectorized");
     return true;
   }
 
   private void validateVectorOperator(Operator<? extends OperatorDesc> vectorOp)
       throws HiveException {
-    if (!vectorOp.getName().equals("TS")) {
-      vectorOp.initialize(job, null);
-    }
+    vectorOp.initialize(job, null);
     if (vectorOp.getChildOperators() != null) {
       for (Operator<? extends OperatorDesc> vop : vectorOp.getChildOperators()) {
         validateVectorOperator(vop);

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java?rev=1481622&r1=1481621&r2=1481622&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/BytesColumnVector.java Sun May 12 18:53:15 2013
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
@@ -202,10 +203,14 @@ public class BytesColumnVector extends C
 
   @Override
   public Writable getWritableObject(int index) {
-    Text result = null;
-    if (!isNull[index]) {
-      result = new Text();
-      result.append(vector[index], start[index], length[index]);
+    if (this.isRepeating) {
+      index = 0;
+    }
+    Writable result = null;
+    if (!isNull[index] && vector[index] != null) {
+      result = new Text(vector[index]);
+    } else {
+      result = NullWritable.get();
     }
     return result;
   }

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java?rev=1481622&r1=1481621&r2=1481622&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/DoubleColumnVector.java Sun May 12 18:53:15 2013
@@ -27,13 +27,13 @@ import org.apache.hadoop.io.Writable;
  * During copy-in/copy-out, smaller types (i.e. float) will be converted as needed. This will
  * reduce the amount of code that needs to be generated and also will run fast since the
  * machine operates with 64-bit words.
- * 
+ *
  * The vector[] field is public by design for high-performance access in the inner
  * loop of query execution.
  */
 public class DoubleColumnVector extends ColumnVector {
   public double[] vector;
-  private DoubleWritable writableObj = new DoubleWritable();
+  private final DoubleWritable writableObj = new DoubleWritable();
 
   /**
    * Use this constructor by default. All column vectors
@@ -42,10 +42,10 @@ public class DoubleColumnVector extends 
   public DoubleColumnVector() {
     this(VectorizedRowBatch.DEFAULT_SIZE);
   }
-  
-  /** 
+
+  /**
    * Don't use this except for testing purposes.
-   * 
+   *
    * @param len
    */
   public DoubleColumnVector(int len) {
@@ -55,6 +55,9 @@ public class DoubleColumnVector extends 
 
   @Override
   public Writable getWritableObject(int index) {
+    if (this.isRepeating) {
+      index = 0;
+    }
     if (!noNulls && isNull[index]) {
       return null;
     } else {

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java?rev=1481622&r1=1481621&r2=1481622&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/LongColumnVector.java Sun May 12 18:53:15 2013
@@ -27,14 +27,14 @@ import org.apache.hadoop.io.Writable;
  * During copy-in/copy-out, smaller int types will be converted as needed. This will
  * reduce the amount of code that needs to be generated and also will run fast since the
  * machine operates with 64-bit words.
- * 
+ *
  * The vector[] field is public by design for high-performance access in the inner
  * loop of query execution.
  */
 public class LongColumnVector extends ColumnVector {
   public long[] vector;
-  private LongWritable writableObj = new LongWritable();
-  
+  private final LongWritable writableObj = new LongWritable();
+
   /**
    * Use this constructor by default. All column vectors
    * should normally be the default size.
@@ -45,7 +45,7 @@ public class LongColumnVector extends Co
 
   /**
    * Don't use this except for testing purposes.
-   * 
+   *
    * @param len
    */
   public LongColumnVector(int len) {
@@ -55,6 +55,9 @@ public class LongColumnVector extends Co
 
   @Override
   public Writable getWritableObject(int index) {
+    if (this.isRepeating) {
+      index = 0;
+    }
     if (!noNulls && isNull[index]) {
       return null;
     } else {

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java?rev=1481622&r1=1481621&r2=1481622&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorFilterOperator.java Sun May 12 18:53:15 2013
@@ -68,7 +68,6 @@ public class VectorFilterOperator extend
       ExprNodeDesc oldExpression = conf.getPredicate();
       vContext.setOperatorType(OperatorType.FILTER);
       conditionEvaluator = vContext.getVectorExpression(oldExpression);
-      System.out.println("Filter class name ="+conditionEvaluator.getClass().getCanonicalName());
       statsMap.put(Counter.FILTERED, filtered_count);
       statsMap.put(Counter.PASSED, passed_count);
     } catch (Throwable e) {

Modified: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java?rev=1481622&r1=1481621&r2=1481622&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java (original)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java Sun May 12 18:53:15 2013
@@ -414,12 +414,12 @@ public class VectorizationContext {
       }
     } else if ( (rightExpr instanceof ExprNodeColumnDesc) &&
         (leftExpr instanceof ExprNodeColumnDesc) ) {
-      ExprNodeColumnDesc rightColDesc = (ExprNodeColumnDesc) rightExpr;
       ExprNodeColumnDesc leftColDesc = (ExprNodeColumnDesc) leftExpr;
-      int inputCol1 = getInputColumnIndex(rightColDesc.getColumn());
-      int inputCol2 = getInputColumnIndex(leftColDesc.getColumn());
-      String colType1 = rightColDesc.getTypeString();
-      String colType2 = leftColDesc.getTypeString();
+      ExprNodeColumnDesc rightColDesc = (ExprNodeColumnDesc) rightExpr;
+      int inputCol1 = getInputColumnIndex(leftColDesc.getColumn());
+      int inputCol2 = getInputColumnIndex(rightColDesc.getColumn());
+      String colType1 = leftColDesc.getTypeString();
+      String colType2 = rightColDesc.getTypeString();
       String outputColType = getOutputColType(colType1, colType2, method);
       String className = getBinaryColumnColumnExpressionClassName(colType1,
           colType2, method);
@@ -686,12 +686,12 @@ public class VectorizationContext {
       }
     } else if ( (rightExpr instanceof ExprNodeColumnDesc) &&
         (leftExpr instanceof ExprNodeColumnDesc) ) {
-      ExprNodeColumnDesc rightColDesc = (ExprNodeColumnDesc) rightExpr;
       ExprNodeColumnDesc leftColDesc = (ExprNodeColumnDesc) leftExpr;
-      int inputCol1 = getInputColumnIndex(rightColDesc.getColumn());
-      int inputCol2 = getInputColumnIndex(leftColDesc.getColumn());
-      String colType1 = rightColDesc.getTypeString();
-      String colType2 = leftColDesc.getTypeString();
+      ExprNodeColumnDesc rightColDesc = (ExprNodeColumnDesc) rightExpr;
+      int inputCol1 = getInputColumnIndex(leftColDesc.getColumn());
+      int inputCol2 = getInputColumnIndex(rightColDesc.getColumn());
+      String colType1 = leftColDesc.getTypeString();
+      String colType2 = rightColDesc.getTypeString();
       String className = getFilterColumnColumnExpressionClassName(colType1,
           colType2, opName);
       try {
@@ -703,15 +703,13 @@ public class VectorizationContext {
     } else if ( (leftExpr instanceof ExprNodeGenericFuncDesc) &&
         (rightExpr instanceof ExprNodeColumnDesc) ) {
       v1 = getVectorExpression((ExprNodeGenericFuncDesc) leftExpr);
-      ExprNodeColumnDesc leftColDesc = (ExprNodeColumnDesc) rightExpr;
+      ExprNodeColumnDesc rightColDesc = (ExprNodeColumnDesc) rightExpr;
       int inputCol1 = v1.getOutputColumn();
-      int inputCol2 = getInputColumnIndex(leftColDesc.getColumn());
+      int inputCol2 = getInputColumnIndex(rightColDesc.getColumn());
       String colType1 = v1.getOutputType();
-      String colType2 = leftColDesc.getTypeString();
+      String colType2 = rightColDesc.getTypeString();
       String className = getFilterColumnColumnExpressionClassName(colType1,
           colType2, opName);
-      System.out.println("In the context, Input column 1: "+inputCol1+
-          ", column 2: "+inputCol2);
       try {
         expr = (VectorExpression) Class.forName(className).
             getDeclaredConstructors()[0].newInstance(inputCol1, inputCol2);
@@ -858,11 +856,18 @@ public class VectorizationContext {
   private String getBinaryColumnScalarExpressionClassName(String colType,
       String scalarType, String method) {
     StringBuilder b = new StringBuilder();
-    b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.gen.");
-    b.append(getNormalizedTypeName(colType));
+    String normColType = getNormalizedTypeName(colType);
+    String normScalarType = getNormalizedTypeName(scalarType);
+    if (normColType.equalsIgnoreCase("long") && normScalarType.equalsIgnoreCase("long")
+        && method.equalsIgnoreCase("divide")) {
+      b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.");
+    } else {
+      b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.gen.");
+    }
+    b.append(normColType);
     b.append("Col");
     b.append(method);
-    b.append(getNormalizedTypeName(scalarType));
+    b.append(normScalarType);
     b.append("Scalar");
     return b.toString();
   }
@@ -870,11 +875,18 @@ public class VectorizationContext {
   private String getBinaryScalarColumnExpressionClassName(String colType,
       String scalarType, String method) {
     StringBuilder b = new StringBuilder();
-    b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.gen.");
-    b.append(this.getNormalizedTypeName(scalarType));
+    String normColType = getNormalizedTypeName(colType);
+    String normScalarType = getNormalizedTypeName(scalarType);
+    if (normColType.equalsIgnoreCase("long") && normScalarType.equalsIgnoreCase("long")
+        && method.equalsIgnoreCase("divide")) {
+      b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.");
+    } else {
+      b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.gen.");
+    }
+    b.append(normScalarType);
     b.append("Scalar");
     b.append(method);
-    b.append(this.getNormalizedTypeName(colType));
+    b.append(normColType);
     b.append("Column");
     return b.toString();
   }
@@ -882,11 +894,18 @@ public class VectorizationContext {
   private String getBinaryColumnColumnExpressionClassName(String colType1,
       String colType2, String method) {
     StringBuilder b = new StringBuilder();
-    b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.gen.");
-    b.append(getNormalizedTypeName(colType1));
+    String normColType1 = getNormalizedTypeName(colType1);
+    String normColType2 = getNormalizedTypeName(colType2);
+    if (normColType1.equalsIgnoreCase("long") && normColType2.equalsIgnoreCase("long")
+        && method.equalsIgnoreCase("divide")) {
+      b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.");
+    } else {
+      b.append("org.apache.hadoop.hive.ql.exec.vector.expressions.gen.");
+    }
+    b.append(normColType1);
     b.append("Col");
     b.append(method);
-    b.append(getNormalizedTypeName(colType2));
+    b.append(normColType2);
     b.append("Column");
     return b.toString();
   }

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java?rev=1481622&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongColumn.java Sun May 12 18:53:15 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;
+
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class LongColDivideLongColumn extends VectorExpression {
+  int colNum1;
+  int colNum2;
+  int outputColumn;
+
+  public LongColDivideLongColumn(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];
+    LongColumnVector inputColVector2 = (LongColumnVector) batch.cols[colNum2];
+    DoubleColumnVector outputColVector = (DoubleColumnVector) batch.cols[outputColumn];
+    int[] sel = batch.selected;
+    int n = batch.size;
+    long[] 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] / (double) 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] / (double) vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[0] / (double) vector2[i];
+        }
+      }
+    } else if (inputColVector2.isRepeating) {
+      if (batch.selectedInUse) {
+        for(int j = 0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] / (double) vector2[0];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] / (double) vector2[0];
+        }
+      }
+    } else {
+      if (batch.selectedInUse) {
+        for(int j=0; j != n; j++) {
+          int i = sel[j];
+          outputVector[i] = vector1[i] / (double) vector2[i];
+        }
+      } else {
+        for(int i = 0; i != n; i++) {
+          outputVector[i] = vector1[i] / (double) 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/LongColDivideLongScalar.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java?rev=1481622&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongColDivideLongScalar.java Sun May 12 18:53:15 2013
@@ -0,0 +1,103 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+
+public class LongColDivideLongScalar extends VectorExpression {
+  private final int colNum;
+  private final double value;
+  private final int outputColumn;
+
+  public LongColDivideLongScalar(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);
+    }
+
+    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";
+  }
+}

Added: hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java?rev=1481622&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java (added)
+++ hive/branches/vectorization/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/LongScalarDivideLongColumn.java Sun May 12 18:53:15 2013
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.exec.vector.expressions;
+
+import org.apache.hadoop.hive.ql.exec.vector.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 final int colNum;
+  private final double value;
+  private final 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) {
+
+    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] = 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";
+  }
+}

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=1481622&r1=1481621&r2=1481622&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 Sun May 12 18:53:15 2013
@@ -14,7 +14,6 @@ public class CodeGen {
       {"ColumnArithmeticScalar", "Add", "long", "long", "+"},
       {"ColumnArithmeticScalar", "Subtract", "long", "long", "-"},
       {"ColumnArithmeticScalar", "Multiply", "long", "long", "*"},
-      {"ColumnArithmeticScalar", "Divide", "long", "long", "/"},
       {"ColumnArithmeticScalar", "Modulo", "long", "long", "%"},
 
       {"ColumnArithmeticScalar", "Add", "long", "double", "+"},
@@ -38,7 +37,6 @@ public class CodeGen {
       {"ScalarArithmeticColumn", "Add", "long", "long", "+"},
       {"ScalarArithmeticColumn", "Subtract", "long", "long", "-"},
       {"ScalarArithmeticColumn", "Multiply", "long", "long", "*"},
-      {"ScalarArithmeticColumn", "Divide", "long", "long", "/"},
       {"ScalarArithmeticColumn", "Modulo", "long", "long", "%"},
 
       {"ScalarArithmeticColumn", "Add", "long", "double", "+"},
@@ -62,7 +60,6 @@ public class CodeGen {
       {"ColumnArithmeticColumn", "Add", "long", "long", "+"},
       {"ColumnArithmeticColumn", "Subtract", "long", "long", "-"},
       {"ColumnArithmeticColumn", "Multiply", "long", "long", "*"},
-      {"ColumnArithmeticColumn", "Divide", "long", "long", "/"},
       {"ColumnArithmeticColumn", "Modulo", "long", "long", "%"},
 
       {"ColumnArithmeticColumn", "Add", "long", "double", "+"},
@@ -154,7 +151,7 @@ public class CodeGen {
       {"FilterStringColumnCompareScalar", "LessEqual", "<="},
       {"FilterStringColumnCompareScalar", "Greater", ">"},
       {"FilterStringColumnCompareScalar", "GreaterEqual", ">="},
-      
+
       {"FilterStringColumnCompareColumn", "Equal", "=="},
       {"FilterStringColumnCompareColumn", "NotEqual", "!="},
       {"FilterStringColumnCompareColumn", "Less", "<"},
@@ -385,7 +382,7 @@ public class CodeGen {
     String className = "FilterStringCol" + operatorName + "StringScalar";
     generateFilterStringColumnCompareScalar(tdesc,className);
   }
-  
+
   private void generateFilterStringColumnCompareColumn(String[] tdesc) throws IOException {
     String operatorName = tdesc[1];
     String className = "FilterStringCol" + operatorName + "StringCol";