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/06/11 23:46:09 UTC

svn commit: r1491980 - in /hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector: FakeVectorRowBatchFromObjectIterables.java TestVectorGroupByOperator.java util/FakeVectorRowBatchFromObjectIterables.java

Author: hashutosh
Date: Tue Jun 11 21:46:08 2013
New Revision: 1491980

URL: http://svn.apache.org/r1491980
Log:
HIVE-4704 : TestCase FakeVectorRowBatchFromObjectIterables error (Eric Hanson via Ashutosh Chauhan)

Added:
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
Removed:
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/FakeVectorRowBatchFromObjectIterables.java
Modified:
    hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java

Modified: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java?rev=1491980&r1=1491979&r2=1491980&view=diff
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java (original)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java Tue Jun 11 21:46:08 2013
@@ -37,6 +37,7 @@ import java.util.Set;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeCaptureOutputOperator;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromConcat;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromLongIterables;
+import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromObjectIterables;
 import org.apache.hadoop.hive.ql.exec.vector.util.FakeVectorRowBatchFromRepeats;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.plan.AggregationDesc;

Added: hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
URL: http://svn.apache.org/viewvc/hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java?rev=1491980&view=auto
==============================================================================
--- hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java (added)
+++ hive/branches/vectorization/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java Tue Jun 11 21:46:08 2013
@@ -0,0 +1,181 @@
+/**
+ * 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.util;
+
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+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.TimestampUtils;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ * Test helper class that creates vectorized execution batches from arbitrary type iterables.
+ */
+public class FakeVectorRowBatchFromObjectIterables extends FakeVectorRowBatchBase {
+
+  private final String[] types;
+  private final List<Iterator<Object>> iterators;
+  private final VectorizedRowBatch batch;
+  private boolean eof;
+  private final int batchSize;
+  
+  public String[] getTypes() {
+    return this.types;
+  }
+
+  /**
+   * Helper interface for assigning values to primitive vector column types.
+   */
+  private static interface ColumnVectorAssign
+  {
+    public void assign(
+        ColumnVector columnVector,
+        int row,
+        Object value);
+  }
+
+  private final ColumnVectorAssign[] columnAssign;
+
+  public FakeVectorRowBatchFromObjectIterables(int batchSize, String[] types,
+      Iterable<Object> ...iterables) throws HiveException {
+    this.types = types;
+    this.batchSize = batchSize;
+    iterators = new ArrayList<Iterator<Object>>(types.length);
+    columnAssign = new ColumnVectorAssign[types.length];
+
+    batch = new VectorizedRowBatch(types.length, batchSize);
+    for(int i=0; i< types.length; ++i) {
+      if (types[i].equalsIgnoreCase("tinyint") ||
+          types[i].equalsIgnoreCase("smallint")||
+          types[i].equalsIgnoreCase("int")||
+          types[i].equalsIgnoreCase("bigint")||
+          types[i].equalsIgnoreCase("long")) {
+        batch.cols[i] = new LongColumnVector(batchSize);
+        columnAssign[i] = new ColumnVectorAssign() {
+          @Override
+          public void assign(
+              ColumnVector columnVector,
+              int row,
+              Object value) {
+            LongColumnVector lcv = (LongColumnVector) columnVector;
+            lcv.vector[row] = Long.valueOf(value.toString());
+          }
+        };
+      } else if (types[i].equalsIgnoreCase("boolean")) {
+        batch.cols[i] = new LongColumnVector(batchSize);
+        columnAssign[i] = new ColumnVectorAssign() {
+          @Override
+          public void assign(
+              ColumnVector columnVector,
+              int row,
+              Object value) {
+            LongColumnVector lcv = (LongColumnVector) columnVector;
+            lcv.vector[row] = (Boolean) value ? 1 : 0;
+          }
+        };
+      } else if (types[i].equalsIgnoreCase("timestamp")) {
+        batch.cols[i] = new LongColumnVector(batchSize);
+        columnAssign[i] = new ColumnVectorAssign() {
+          @Override
+          public void assign(
+              ColumnVector columnVector,
+              int row,
+              Object value) {
+            LongColumnVector lcv = (LongColumnVector) columnVector;
+            Timestamp t = (Timestamp) value;
+            lcv.vector[row] = TimestampUtils.getTimeNanoSec(t);
+          }
+        };
+        
+      } else if (types[i].equalsIgnoreCase("string")) {
+        batch.cols[i] = new BytesColumnVector(batchSize);
+        columnAssign[i] = new ColumnVectorAssign() {
+          @Override
+          public void assign(
+              ColumnVector columnVector,
+              int row,
+              Object value) {
+            BytesColumnVector bcv = (BytesColumnVector) columnVector;
+            String s = (String) value;
+            byte[] bytes = s.getBytes();
+            bcv.vector[row] = bytes;
+            bcv.start[row] = 0;
+            bcv.length[row] = bytes.length;
+          }
+        };
+      } else if (types[i].equalsIgnoreCase("double") ||
+          types[i].equalsIgnoreCase("float")) {
+        batch.cols[i] = new DoubleColumnVector(batchSize);
+        columnAssign[i] = new ColumnVectorAssign() {
+          @Override
+          public void assign(
+              ColumnVector columnVector,
+              int row,
+              Object value) {
+            DoubleColumnVector dcv = (DoubleColumnVector) columnVector;
+            dcv.vector[row] = Double.valueOf(value.toString());
+          }
+        };
+      } else {
+        throw new HiveException("Unimplemented type " + types[i]);
+      }
+      iterators.add(iterables[i].iterator());
+    }
+  }
+
+  @Override
+  public VectorizedRowBatch produceNextBatch() {
+    batch.size = 0;
+    batch.selectedInUse = false;
+    for (int i=0; i < types.length; ++i) {
+      ColumnVector col = batch.cols[i];
+      col.noNulls = true;
+      col.isRepeating = false;
+    }
+    while (!eof && batch.size < this.batchSize){
+      int r = batch.size;
+      for (int i=0; i < types.length; ++i) {
+        Iterator<Object> it = iterators.get(i);
+        if (!it.hasNext()) {
+          eof = true;
+          break;
+        }
+        Object value = it.next();
+        if (null == value) {
+          batch.cols[i].isNull[batch.size] = true;
+          batch.cols[i].noNulls = false;
+        } else {
+          columnAssign[i].assign(batch.cols[i], batch.size, value);
+        }
+      }
+      if (!eof) {
+        batch.size += 1;
+      }
+    }
+    return batch;
+  }
+}
+