You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by go...@apache.org on 2017/04/27 03:10:48 UTC

hive git commit: HIVE-16533: Vectorization: Avoid evaluating empty groupby keys (Gopal V, reviewed by Sergey Shelukhin)

Repository: hive
Updated Branches:
  refs/heads/master 1cae318fb -> e35b3f253


HIVE-16533: Vectorization: Avoid evaluating empty groupby keys (Gopal V, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/e35b3f25
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/e35b3f25
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/e35b3f25

Branch: refs/heads/master
Commit: e35b3f253d87fdc5120e733a8654b2aa85884e4e
Parents: 1cae318
Author: Gopal V <go...@apache.org>
Authored: Wed Apr 26 20:10:30 2017 -0700
Committer: Gopal V <go...@apache.org>
Committed: Wed Apr 26 20:10:30 2017 -0700

----------------------------------------------------------------------
 .../hadoop/hive/ql/exec/vector/VectorGroupByOperator.java | 10 +++++++++-
 .../hive/ql/exec/vector/VectorHashKeyWrapperBatch.java    |  6 ++++++
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e35b3f25/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
index 4b76d74..5b4c7c3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
@@ -458,10 +458,18 @@ public class VectorGroupByOperator extends Operator<GroupByDesc> implements
       // to bump its internal version.
       aggregationBatchInfo.startBatch();
 
+      if (batch.size == 0) {
+        return;
+      }
+
       // We now have to probe the global hash and find-or-allocate
       // the aggregation buffers to use for each key present in the batch
       VectorHashKeyWrapper[] keyWrappers = keyWrappersBatch.getVectorHashKeyWrappers();
-      for (int i=0; i < batch.size; ++i) {
+
+      final int n = keyExpressions.length == 0 ? 1 : batch.size;
+      // note - the row mapping is not relevant when aggregationBatchInfo::getDistinctBufferSetCount() == 1
+
+      for (int i=0; i < n; ++i) {
         VectorHashKeyWrapper kw = keyWrappers[i];
         VectorAggregationBufferRow aggregationBuffer = mapKeysAggregationBuffers.get(kw);
         if (null == aggregationBuffer) {

http://git-wip-us.apache.org/repos/asf/hive/blob/e35b3f25/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
index 63cdf94..b235a3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.exec.vector;
 
+import org.apache.hadoop.hive.ql.exec.vector.VectorHashKeyWrapper.EmptyVectorHashKeyWrapper;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -89,6 +90,11 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
    */
   public void evaluateBatch(VectorizedRowBatch batch) throws HiveException {
 
+    if (keyCount == 0) {
+      // all keywrappers must be EmptyVectorHashKeyWrapper
+      return;
+    }
+
     for(int i=0;i<batch.size;++i) {
       vectorHashKeyWrappers[i].clearIsNull();
     }