HIVE-14016 : Vectorization: Add support for Grouping Sets (Matt McCline, 
reviewed by Gopal Vijayaraghavan, Jesus Camacho Rodriguez)


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

Branch: refs/heads/master
Commit: 47ef02e4877690cd6e5a4ef5f53cb55c41b8803e
Parents: 62ea542
Author: Matt McCline <mmccl...@hortonworks.com>
Authored: Sun Mar 19 01:54:08 2017 -0500
Committer: Matt McCline <mmccl...@hortonworks.com>
Committed: Sun Mar 19 01:54:08 2017 -0500

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |  14 +
 .../ql/exec/vector/VectorColumnSetInfo.java     | 158 ++--
 .../ql/exec/vector/VectorGroupByOperator.java   | 144 +++-
 .../ql/exec/vector/VectorGroupKeyHelper.java    |  15 +-
 .../ql/exec/vector/VectorHashKeyWrapper.java    | 200 +++--
 .../exec/vector/VectorHashKeyWrapperBatch.java  | 445 +++++++---
 .../ql/exec/vector/VectorMapJoinOperator.java   |   3 +
 .../exec/vector/VectorSMBMapJoinOperator.java   |   3 +
 .../hive/ql/optimizer/physical/Vectorizer.java  |  19 +-
 .../clientpositive/groupby_grouping_id1.q       |   2 +
 .../clientpositive/groupby_grouping_id2.q       |   4 +
 .../clientpositive/groupby_grouping_id3.q       |  12 +
 .../clientpositive/groupby_grouping_sets1.q     |  20 +
 .../clientpositive/groupby_grouping_sets2.q     |   4 +
 .../clientpositive/groupby_grouping_sets3.q     |   6 +
 .../clientpositive/groupby_grouping_sets4.q     |   1 +
 .../clientpositive/groupby_grouping_sets5.q     |   2 +
 .../clientpositive/groupby_grouping_sets6.q     |   2 +
 .../groupby_grouping_sets_grouping.q            |   3 +
 .../groupby_grouping_sets_limit.q               |   3 +
 .../clientpositive/vector_groupby_cube1.q       |  55 ++
 .../vector_groupby_grouping_id1.q               |  23 +
 .../vector_groupby_grouping_id2.q               |  65 ++
 .../vector_groupby_grouping_id3.q               |  42 +
 .../vector_groupby_grouping_sets1.q             |  43 +
 .../vector_groupby_grouping_sets2.q             |  36 +
 .../vector_groupby_grouping_sets3.q             |  40 +
 .../vector_groupby_grouping_sets4.q             |  57 ++
 .../vector_groupby_grouping_sets5.q             |  39 +
 .../vector_groupby_grouping_sets6.q             |  38 +
 .../vector_groupby_grouping_sets_grouping.q     |  99 +++
 .../vector_groupby_grouping_sets_limit.q        |  43 +
 .../vector_groupby_grouping_window.q            |  21 +
 .../clientpositive/vector_groupby_rollup1.q     |  54 ++
 .../clientpositive/groupby_grouping_id1.q.out   | 120 +--
 .../clientpositive/groupby_grouping_id3.q.out   | 139 +++
 .../clientpositive/groupby_grouping_sets1.q.out | 496 ++++++++++-
 .../clientpositive/groupby_grouping_sets2.q.out |  62 +-
 .../clientpositive/groupby_grouping_sets3.q.out |  41 +-
 .../clientpositive/groupby_grouping_sets5.q.out |  36 +-
 .../clientpositive/groupby_grouping_sets6.q.out |   4 +-
 .../groupby_grouping_sets_grouping.q.out        | 104 +--
 .../groupby_grouping_sets_limit.q.out           |  34 +-
 .../llap/groupby_grouping_id2.q.out             |   9 +
 .../llap/vector_empty_where.q.out               |  16 +-
 .../llap/vector_groupby_cube1.q.out             | 773 +++++++++++++++++
 .../llap/vector_groupby_grouping_id1.q.out      | 179 ++++
 .../llap/vector_groupby_grouping_id2.q.out      | 359 ++++++++
 .../llap/vector_groupby_grouping_id3.q.out      | 232 +++++
 .../llap/vector_groupby_grouping_sets1.q.out    | 668 +++++++++++++++
 .../llap/vector_groupby_grouping_sets2.q.out    | 469 +++++++++++
 .../llap/vector_groupby_grouping_sets3.q.out    | 314 +++++++
 .../llap/vector_groupby_grouping_sets4.q.out    | 554 ++++++++++++
 .../llap/vector_groupby_grouping_sets5.q.out    | 371 ++++++++
 .../llap/vector_groupby_grouping_sets6.q.out    | 192 +++++
 .../vector_groupby_grouping_sets_grouping.q.out | 838 +++++++++++++++++++
 .../vector_groupby_grouping_sets_limit.q.out    | 568 +++++++++++++
 .../llap/vector_groupby_grouping_window.q.out   | 157 ++++
 .../llap/vector_groupby_rollup1.q.out           | 610 ++++++++++++++
 .../llap/vector_grouping_sets.q.out             |  69 +-
 .../llap/vector_non_string_partition.q.out      | 144 +++-
 .../llap/vector_tablesample_rows.q.out          |   2 +-
 .../clientpositive/llap/vectorization_15.q.out  |   6 +-
 .../llap/vectorization_div0.q.out               | 414 +++++----
 .../llap/vectorization_limit.q.out              | 470 +++++++++--
 .../llap/vectorization_offset_limit.q.out       |   2 +-
 .../llap/vectorized_mapjoin2.q.out              |   4 +-
 .../spark/groupby_grouping_id2.q.out            |   9 +
 .../clientpositive/spark/vectorization_15.q.out |   6 +-
 .../clientpositive/vector_grouping_sets.q.out   |  51 +-
 .../clientpositive/vectorization_15.q.out       |   6 +-
 71 files changed, 9396 insertions(+), 847 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties 
b/itests/src/test/resources/testconfiguration.properties
index e1ae8fb..afcc3ee 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -612,6 +612,20 @@ minillaplocal.query.files=acid_globallimit.q,\
   vector_auto_smb_mapjoin_14.q,\
   vector_decimal_2.q,\
   vector_decimal_udf.q,\
+  vector_groupby_cube1.q,\
+  vector_groupby_grouping_id1.q,\
+  vector_groupby_grouping_id2.q,\
+  vector_groupby_grouping_id3.q,\
+  vector_groupby_grouping_sets1.q,\
+  vector_groupby_grouping_sets2.q,\
+  vector_groupby_grouping_sets3.q,\
+  vector_groupby_grouping_sets4.q,\
+  vector_groupby_grouping_sets5.q,\
+  vector_groupby_grouping_sets6.q,\
+  vector_groupby_grouping_sets_grouping.q,\
+  vector_groupby_grouping_sets_limit.q,\
+  vector_groupby_grouping_window.q,\
+  vector_groupby_rollup1.q,\
   vector_join30.q,\
   vector_join_filters.q,\
   vector_leftsemi_mapjoin.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
index 935b47b..7ac4f07 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorColumnSetInfo.java
@@ -20,10 +20,8 @@ package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.util.Arrays;
 
-import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
 /**
  * Class to keep information on a set of typed vector columns.  Used by
@@ -64,147 +62,87 @@ public class VectorColumnSetInfo {
    */
   protected int[] intervalDayTimeIndices;
 
-  /**
-   * Helper class for looking up a key value based on key index.
-   */
-  public class KeyLookupHelper {
-    public int longIndex;
-    public int doubleIndex;
-    public int stringIndex;
-    public int decimalIndex;
-    public int timestampIndex;
-    public int intervalDayTimeIndex;
-
-    private static final int INDEX_UNUSED = -1;
-
-    private void resetIndices() {
-        this.longIndex = this.doubleIndex = this.stringIndex = 
this.decimalIndex =
-            timestampIndex = intervalDayTimeIndex = INDEX_UNUSED;
-    }
-    public void setLong(int index) {
-      resetIndices();
-      this.longIndex= index;
-    }
-
-    public void setDouble(int index) {
-      resetIndices();
-      this.doubleIndex = index;
-    }
-
-    public void setString(int index) {
-      resetIndices();
-      this.stringIndex = index;
-    }
-
-    public void setDecimal(int index) {
-      resetIndices();
-      this.decimalIndex = index;
-    }
-
-    public void setTimestamp(int index) {
-      resetIndices();
-      this.timestampIndex= index;
-    }
-
-    public void setIntervalDayTime(int index) {
-      resetIndices();
-      this.intervalDayTimeIndex= index;
-    }
-  }
-
-  /**
-   * Lookup vector to map from key index to primitive type index.
-   */
-  protected KeyLookupHelper[] indexLookup;
+  final protected int keyCount;
+  private int addKeyIndex;
 
-  private int keyCount;
-  private int addIndex;
+  private int addLongIndex;
+  private int addDoubleIndex;
+  private int addStringIndex;
+  private int addDecimalIndex;
+  private int addTimestampIndex;
+  private int addIntervalDayTimeIndex;
 
-  protected int longIndicesIndex;
-  protected int doubleIndicesIndex;
-  protected int stringIndicesIndex;
-  protected int decimalIndicesIndex;
-  protected int timestampIndicesIndex;
-  protected int intervalDayTimeIndicesIndex;
+  // Given the keyIndex these arrays return:
+  //   The ColumnVector.Type,
+  //   The type specific index into longIndices, doubleIndices, etc...
+  protected ColumnVector.Type[] columnVectorTypes;
+  protected int[] columnTypeSpecificIndices;
 
   protected VectorColumnSetInfo(int keyCount) {
     this.keyCount = keyCount;
-    this.addIndex = 0;
+    this.addKeyIndex = 0;
 
     // We'll over allocate and then shrink the array for each type
     longIndices = new int[this.keyCount];
-    longIndicesIndex = 0;
+    addLongIndex = 0;
     doubleIndices = new int[this.keyCount];
-    doubleIndicesIndex  = 0;
+    addDoubleIndex  = 0;
     stringIndices = new int[this.keyCount];
-    stringIndicesIndex = 0;
+    addStringIndex = 0;
     decimalIndices = new int[this.keyCount];
-    decimalIndicesIndex = 0;
+    addDecimalIndex = 0;
     timestampIndices = new int[this.keyCount];
-    timestampIndicesIndex = 0;
+    addTimestampIndex = 0;
     intervalDayTimeIndices = new int[this.keyCount];
-    intervalDayTimeIndicesIndex = 0;
-    indexLookup = new KeyLookupHelper[this.keyCount];
-  }
+    addIntervalDayTimeIndex = 0;
 
-  protected void addKey(String outputType) throws HiveException {
-    indexLookup[addIndex] = new KeyLookupHelper();
+    columnVectorTypes = new ColumnVector.Type[this.keyCount];
+    columnTypeSpecificIndices = new int[this.keyCount];
+  }
 
-    String typeName = VectorizationContext.mapTypeNameSynonyms(outputType);
 
-    TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
-    Type columnVectorType = 
VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
+  protected void addKey(ColumnVector.Type columnVectorType) throws 
HiveException {
 
     switch (columnVectorType) {
     case LONG:
-      longIndices[longIndicesIndex] = addIndex;
-      indexLookup[addIndex].setLong(longIndicesIndex);
-      ++longIndicesIndex;
+      longIndices[addLongIndex] = addKeyIndex;
+      columnTypeSpecificIndices[addKeyIndex] = addLongIndex++;
       break;
-
     case DOUBLE:
-      doubleIndices[doubleIndicesIndex] = addIndex;
-      indexLookup[addIndex].setDouble(doubleIndicesIndex);
-      ++doubleIndicesIndex;
+      doubleIndices[addDoubleIndex] = addKeyIndex;
+      columnTypeSpecificIndices[addKeyIndex] = addDoubleIndex++;
       break;
-
     case BYTES:
-      stringIndices[stringIndicesIndex]= addIndex;
-      indexLookup[addIndex].setString(stringIndicesIndex);
-      ++stringIndicesIndex;
+      stringIndices[addStringIndex]= addKeyIndex;
+      columnTypeSpecificIndices[addKeyIndex] = addStringIndex++;
       break;
-
     case DECIMAL:
-      decimalIndices[decimalIndicesIndex]= addIndex;
-      indexLookup[addIndex].setDecimal(decimalIndicesIndex);
-      ++decimalIndicesIndex;
-      break;
-
+      decimalIndices[addDecimalIndex]= addKeyIndex;
+      columnTypeSpecificIndices[addKeyIndex] = addDecimalIndex++;
+        break;
     case TIMESTAMP:
-      timestampIndices[timestampIndicesIndex] = addIndex;
-      indexLookup[addIndex].setTimestamp(timestampIndicesIndex);
-      ++timestampIndicesIndex;
+      timestampIndices[addTimestampIndex] = addKeyIndex;
+      columnTypeSpecificIndices[addKeyIndex] = addTimestampIndex++;
       break;
-
     case INTERVAL_DAY_TIME:
-      intervalDayTimeIndices[intervalDayTimeIndicesIndex] = addIndex;
-      indexLookup[addIndex].setIntervalDayTime(intervalDayTimeIndicesIndex);
-      ++intervalDayTimeIndicesIndex;
+      intervalDayTimeIndices[addIntervalDayTimeIndex] = addKeyIndex;
+      columnTypeSpecificIndices[addKeyIndex] = addIntervalDayTimeIndex++;
       break;
-
     default:
       throw new HiveException("Unexpected column vector type " + 
columnVectorType);
     }
 
-    addIndex++;
+    columnVectorTypes[addKeyIndex] = columnVectorType;
+    addKeyIndex++;
   }
 
-  protected void finishAdding() {
-    longIndices = Arrays.copyOf(longIndices, longIndicesIndex);
-    doubleIndices = Arrays.copyOf(doubleIndices, doubleIndicesIndex);
-    stringIndices = Arrays.copyOf(stringIndices, stringIndicesIndex);
-    decimalIndices = Arrays.copyOf(decimalIndices, decimalIndicesIndex);
-    timestampIndices = Arrays.copyOf(timestampIndices, timestampIndicesIndex);
-    intervalDayTimeIndices = Arrays.copyOf(intervalDayTimeIndices, 
intervalDayTimeIndicesIndex);
+
+  protected void finishAdding() throws HiveException {
+    longIndices = Arrays.copyOf(longIndices, addLongIndex);
+    doubleIndices = Arrays.copyOf(doubleIndices, addDoubleIndex);
+    stringIndices = Arrays.copyOf(stringIndices, addStringIndex);
+    decimalIndices = Arrays.copyOf(decimalIndices, addDecimalIndex);
+    timestampIndices = Arrays.copyOf(timestampIndices, addTimestampIndex);
+    intervalDayTimeIndices = Arrays.copyOf(intervalDayTimeIndices, 
addIntervalDayTimeIndex);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/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 fef7c2a..90748e9 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
@@ -22,16 +22,21 @@ import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
 import java.lang.ref.SoftReference;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
+import org.apache.hadoop.hive.ql.exec.GroupByOperator;
 import org.apache.hadoop.hive.ql.exec.KeyWrapper;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import 
org.apache.hadoop.hive.ql.exec.vector.expressions.ConstantVectorExpression;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.IdentityExpression;
 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.exec.vector.expressions.VectorExpressionWriterFactory;
@@ -52,6 +57,8 @@ import org.apache.hadoop.io.DataOutputBuffer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javolution.util.FastBitSet;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 
@@ -110,6 +117,24 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
 
   private transient VectorAssignRow vectorAssignRow;
 
+  /*
+   * Grouping sets members.
+   */
+  private transient boolean groupingSetsPresent;
+
+  // The field bits (i.e. which fields to include) or "id" for each grouping 
set.
+  private transient int[] groupingSets;
+
+  // The position in the column keys of the dummy grouping set id column.
+  private transient int groupingSetsPosition;
+
+  // The planner puts a constant field in for the dummy grouping set id.  We 
will overwrite it
+  // as we process the grouping sets.
+  private transient ConstantVectorExpression groupingSetsDummyVectorExpression;
+
+  // We translate the grouping set bit field into a boolean arrays.
+  private transient boolean[][] allGroupingSetsOverrideIsNulls;
+
   private transient int numEntriesHashTable;
 
   private transient long maxHashTblMemory;
@@ -144,6 +169,32 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
       // Do nothing.
     }
 
+    protected abstract void doProcessBatch(VectorizedRowBatch batch, boolean 
isFirstGroupingSet,
+        boolean[] currentGroupingSetsOverrideIsNulls) throws HiveException;
+
+    @Override
+    public void processBatch(VectorizedRowBatch batch) throws HiveException {
+
+      if (!groupingSetsPresent) {
+        doProcessBatch(batch, false, null);
+        return;
+      }
+
+      // We drive the doProcessBatch logic with the same batch but different
+      // grouping set id and null variation.
+      // PERFORMANCE NOTE: We do not try to reuse columns and generate the 
KeyWrappers anew...
+
+      final int size = groupingSets.length;
+      for (int i = 0; i < size; i++) {
+
+        // NOTE: We are overwriting the constant vector value...
+        groupingSetsDummyVectorExpression.setLongValue(groupingSets[i]);
+        groupingSetsDummyVectorExpression.evaluate(batch);
+
+        doProcessBatch(batch, (i == 0), allGroupingSetsOverrideIsNulls[i]);
+      }
+    }
+
     /**
      * Evaluates the aggregators on the current batch.
      * The aggregationBatchInfo must have been prepared
@@ -207,7 +258,8 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
     }
 
     @Override
-    public void processBatch(VectorizedRowBatch batch) throws HiveException {
+    public void doProcessBatch(VectorizedRowBatch batch, boolean 
isFirstGroupingSet,
+        boolean[] currentGroupingSetsOverrideIsNulls) throws HiveException {
       for (int i = 0; i < aggregators.length; ++i) {
         
aggregators[i].aggregateInput(aggregationBuffers.getAggregationBuffer(i), 
batch);
       }
@@ -328,11 +380,24 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
     }
 
     @Override
-    public void processBatch(VectorizedRowBatch batch) throws HiveException {
+    public void doProcessBatch(VectorizedRowBatch batch, boolean 
isFirstGroupingSet,
+        boolean[] currentGroupingSetsOverrideIsNulls) throws HiveException {
+
+      if (!groupingSetsPresent || isFirstGroupingSet) {
+
+        // Evaluate the key expressions once.
+        for(int i = 0; i < keyExpressions.length; ++i) {
+          keyExpressions[i].evaluate(batch);
+        }
+      }
 
       // First we traverse the batch to evaluate and prepare the KeyWrappers
       // After this the KeyWrappers are properly set and hash code is computed
-      keyWrappersBatch.evaluateBatch(batch);
+      if (!groupingSetsPresent) {
+        keyWrappersBatch.evaluateBatch(batch);
+      } else {
+        keyWrappersBatch.evaluateBatchGroupingSets(batch, 
currentGroupingSetsOverrideIsNulls);
+      }
 
       // Next we locate the aggregation buffer set for each key
       prepareBatchAggregationBufferSets(batch);
@@ -607,10 +672,24 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
     }
 
     @Override
-    public void processBatch(VectorizedRowBatch batch) throws HiveException {
+    public void doProcessBatch(VectorizedRowBatch batch, boolean 
isFirstGroupingSet,
+        boolean[] currentGroupingSetsOverrideIsNulls) throws HiveException {
+
+      if (!groupingSetsPresent || isFirstGroupingSet) {
+
+        // Evaluate the key expressions once.
+        for(int i = 0; i < keyExpressions.length; ++i) {
+          keyExpressions[i].evaluate(batch);
+        }
+      }
+
       // First we traverse the batch to evaluate and prepare the KeyWrappers
       // After this the KeyWrappers are properly set and hash code is computed
-      keyWrappersBatch.evaluateBatch(batch);
+      if (!groupingSetsPresent) {
+        keyWrappersBatch.evaluateBatch(batch);
+      } else {
+        keyWrappersBatch.evaluateBatchGroupingSets(batch, 
currentGroupingSetsOverrideIsNulls);
+      }
 
       VectorHashKeyWrapper[] batchKeys = 
keyWrappersBatch.getVectorHashKeyWrappers();
 
@@ -702,7 +781,10 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
     @Override
     public void initialize(Configuration hconf) throws HiveException {
       inGroup = false;
-      groupKeyHelper = new VectorGroupKeyHelper(keyExpressions.length);
+
+      // We do not include the dummy grouping set column in the output.  So we 
pass outputKeyLength
+      // instead of keyExpressions.length
+      groupKeyHelper = new VectorGroupKeyHelper(outputKeyLength);
       groupKeyHelper.init(keyExpressions);
       groupAggregators = allocateAggregationBuffer();
       buffer = new DataOutputBuffer();
@@ -725,7 +807,8 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
     }
 
     @Override
-    public void processBatch(VectorizedRowBatch batch) throws HiveException {
+    public void doProcessBatch(VectorizedRowBatch batch, boolean 
isFirstGroupingSet,
+        boolean[] currentGroupingSetsOverrideIsNulls) throws HiveException {
       assert(inGroup);
       if (first) {
         // Copy the group key to output batch now.  We'll copy in the 
aggregates at the end of the group.
@@ -778,6 +861,49 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
     super(ctx);
   }
 
+  private void setupGroupingSets() {
+
+    groupingSetsPresent = conf.isGroupingSetsPresent();
+    if (!groupingSetsPresent) {
+      groupingSets = null;
+      groupingSetsPosition = -1;
+      groupingSetsDummyVectorExpression = null;
+      allGroupingSetsOverrideIsNulls = null;
+      return;
+    }
+
+    groupingSets = 
ArrayUtils.toPrimitive(conf.getListGroupingSets().toArray(new Integer[0]));
+    groupingSetsPosition = conf.getGroupingSetPosition();
+
+    allGroupingSetsOverrideIsNulls = new boolean[groupingSets.length][];
+
+    int pos = 0;
+    for (int groupingSet: groupingSets) {
+
+      // Create the mapping corresponding to the grouping set
+
+      // Assume all columns are null, except the dummy column is always 
non-null.
+      boolean[] groupingSetsOverrideIsNull = new 
boolean[keyExpressions.length];
+      Arrays.fill(groupingSetsOverrideIsNull, true);
+      groupingSetsOverrideIsNull[groupingSetsPosition] = false;
+
+      // Add keys of this grouping set.
+      FastBitSet bitset = GroupByOperator.groupingSet2BitSet(groupingSet, 
groupingSetsPosition);
+      for (int keyPos = bitset.nextClearBit(0); keyPos < groupingSetsPosition;
+        keyPos = bitset.nextClearBit(keyPos+1)) {
+        groupingSetsOverrideIsNull[keyPos] = false;
+      }
+
+      allGroupingSetsOverrideIsNulls[pos] =  groupingSetsOverrideIsNull;
+      pos++;
+    }
+
+    // The last key column is the dummy grouping set id.
+    //
+    // Figure out which (scratch) column was used so we can overwrite the 
dummy id.
+
+    groupingSetsDummyVectorExpression = (ConstantVectorExpression) 
keyExpressions[groupingSetsPosition];
+  }
 
   @Override
   protected void initializeOp(Configuration hconf) throws HiveException {
@@ -831,15 +957,19 @@ public class VectorGroupByOperator extends 
Operator<GroupByDesc> implements
 
     forwardCache = new Object[outputKeyLength + aggregators.length];
 
+    setupGroupingSets();
+
     switch (vectorDesc.getProcessingMode()) {
     case GLOBAL:
       Preconditions.checkState(outputKeyLength == 0);
+      Preconditions.checkState(!groupingSetsPresent);
       processingMode = this.new ProcessingModeGlobalAggregate();
       break;
     case HASH:
       processingMode = this.new ProcessingModeHashAggregate();
       break;
     case MERGE_PARTIAL:
+      Preconditions.checkState(!groupingSetsPresent);
       processingMode = this.new ProcessingModeReduceMergePartial();
       break;
     case STREAMING:

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
index 50d0452..0ff389e 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupKeyHelper.java
@@ -19,8 +19,12 @@
 package org.apache.hadoop.hive.ql.exec.vector;
 
 import java.io.IOException;
+
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 import org.apache.hadoop.io.DataOutputBuffer;
 
 /**
@@ -33,9 +37,16 @@ public class VectorGroupKeyHelper extends 
VectorColumnSetInfo {
    }
 
   void init(VectorExpression[] keyExpressions) throws HiveException {
+
+    // NOTE: To support pruning the grouping set id dummy key by 
VectorGroupbyOpeator MERGE_PARTIAL
+    // case, we use the keyCount passed to the constructor and not 
keyExpressions.length.
+
     // Inspect the output type of each key expression.
-    for(int i=0; i < keyExpressions.length; ++i) {
-      addKey(keyExpressions[i].getOutputType());
+    for(int i=0; i < keyCount; ++i) {
+      String typeName = 
VectorizationContext.mapTypeNameSynonyms(keyExpressions[i].getOutputType());
+      TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+      Type columnVectorType = 
VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
+      addKey(columnVectorType);
     }
     finishAdding();
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
index 5de59b1..2c51882 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
+import com.google.common.base.Preconditions;
+
 /**
  * A hash map key wrapper for vectorized processing.
  * It stores the key values as primitives in arrays for each supported 
primitive type.
@@ -59,15 +61,22 @@ public class VectorHashKeyWrapper extends KeyWrapper {
   private HiveDecimalWritable[] decimalValues;
 
   private Timestamp[] timestampValues;
+  private static Timestamp ZERO_TIMESTAMP = new Timestamp(0);
 
   private HiveIntervalDayTime[] intervalDayTimeValues;
+  private static HiveIntervalDayTime ZERO_INTERVALDAYTIME= new 
HiveIntervalDayTime(0, 0);
 
+  // NOTE: The null array is indexed by keyIndex, which is not available 
internally.  The mapping
+  //       from a long, double, etc index to key index is kept once in the 
separate
+  //       VectorColumnSetInfo object.
   private boolean[] isNull;
+
   private int hashcode;
 
   private VectorHashKeyWrapper(int longValuesCount, int doubleValuesCount,
           int byteValuesCount, int decimalValuesCount, int 
timestampValuesCount,
-          int intervalDayTimeValuesCount) {
+          int intervalDayTimeValuesCount,
+          int keyCount) {
     longValues = longValuesCount > 0 ? new long[longValuesCount] : 
EMPTY_LONG_ARRAY;
     doubleValues = doubleValuesCount > 0 ? new double[doubleValuesCount] : 
EMPTY_DOUBLE_ARRAY;
     decimalValues = decimalValuesCount > 0 ? new 
HiveDecimalWritable[decimalValuesCount] : EMPTY_DECIMAL_ARRAY;
@@ -91,8 +100,7 @@ public class VectorHashKeyWrapper extends KeyWrapper {
     for(int i = 0; i < intervalDayTimeValuesCount; ++i) {
       intervalDayTimeValues[i] = new HiveIntervalDayTime();
     }
-    isNull = new boolean[longValuesCount + doubleValuesCount + byteValuesCount 
+
-                         decimalValuesCount + timestampValuesCount + 
intervalDayTimeValuesCount];
+    isNull = new boolean[keyCount];
     hashcode = 0;
   }
 
@@ -101,13 +109,14 @@ public class VectorHashKeyWrapper extends KeyWrapper {
 
   public static VectorHashKeyWrapper allocate(int longValuesCount, int 
doubleValuesCount,
       int byteValuesCount, int decimalValuesCount, int timestampValuesCount,
-      int intervalDayTimeValuesCount) {
+      int intervalDayTimeValuesCount, int keyCount) {
     if ((longValuesCount + doubleValuesCount + byteValuesCount + 
decimalValuesCount
         + timestampValuesCount + intervalDayTimeValuesCount) == 0) {
       return EMPTY_KEY_WRAPPER;
     }
     return new VectorHashKeyWrapper(longValuesCount, doubleValuesCount, 
byteValuesCount,
-        decimalValuesCount, timestampValuesCount, intervalDayTimeValuesCount);
+        decimalValuesCount, timestampValuesCount, intervalDayTimeValuesCount,
+        keyCount);
   }
 
   @Override
@@ -140,19 +149,14 @@ public class VectorHashKeyWrapper extends KeyWrapper {
        *  Hashing the string is potentially expensive so is better to branch.
        *  Additionally not looking at values for nulls allows us not reset the 
values.
        */
-      if (!isNull[longValues.length + doubleValues.length + i]) {
+      if (byteLengths[i] != -1) {
         byte[] bytes = byteValues[i];
         int start = byteStarts[i];
         int length = byteLengths[i];
-        if (length == bytes.length && start == 0) {
-          hashcode ^= Arrays.hashCode(bytes);
-        }
-        else {
-          // Unfortunately there is no Arrays.hashCode(byte[], start, length)
-          for(int j = start; j < start + length; ++j) {
-            // use 461 as is a (sexy!) prime.
-            hashcode ^= 461 * bytes[j];
-          }
+        // Unfortunately there is no Arrays.hashCode(byte[], start, length)
+        for(int j = start; j < start + length; ++j) {
+          // use 461 as is a (sexy!) prime.
+          hashcode ^= 461 * bytes[j];
         }
       }
     }
@@ -184,7 +188,7 @@ public class VectorHashKeyWrapper extends KeyWrapper {
     //By the time we enter here the byteValues.lentgh and isNull must have 
already been compared
     for (int i = 0; i < byteValues.length; ++i) {
       // the byte comparison is potentially expensive so is better to branch 
on null
-      if (!isNull[longValues.length + doubleValues.length + i]) {
+      if (byteLengths[i] != -1) {
         if (!StringExpr.equal(
             byteValues[i],
             byteStarts[i],
@@ -228,7 +232,7 @@ public class VectorHashKeyWrapper extends KeyWrapper {
       for (int i = 0; i < byteValues.length; ++i) {
         // avoid allocation/copy of nulls, because it potentially expensive.
         // branch instead.
-        if (!isNull[longValues.length + doubleValues.length + i]) {
+        if (byteLengths[i] != -1) {
           clone.byteValues[i] = Arrays.copyOfRange(byteValues[i],
               byteStarts[i], byteStarts[i] + byteLengths[i]);
         }
@@ -274,106 +278,141 @@ public class VectorHashKeyWrapper extends KeyWrapper {
     throw new UnsupportedOperationException();
   }
 
-  public void assignDouble(int index, double d) {
-    doubleValues[index] = d;
-    isNull[longValues.length + index] = false;
+  public void assignLong(int index, long v) {
+    longValues[index] = v;
   }
 
-  public void assignNullDouble(int index) {
-    doubleValues[index] = 0; // assign 0 to simplify hashcode
-    isNull[longValues.length + index] = true;
+  public void assignNullLong(int keyIndex, int index) {
+    isNull[keyIndex] = true;
+    longValues[index] = 0; // assign 0 to simplify hashcode
   }
 
-  public void assignLong(int index, long v) {
-    longValues[index] = v;
-    isNull[index] = false;
+  public void assignDouble(int index, double d) {
+    doubleValues[index] = d;
   }
 
-  public void assignNullLong(int index) {
-    longValues[index] = 0; // assign 0 to simplify hashcode
-    isNull[index] = true;
+  public void assignNullDouble(int keyIndex, int index) {
+    isNull[keyIndex] = true;
+    doubleValues[index] = 0; // assign 0 to simplify hashcode
   }
 
   public void assignString(int index, byte[] bytes, int start, int length) {
+    Preconditions.checkState(bytes != null);
     byteValues[index] = bytes;
     byteStarts[index] = start;
     byteLengths[index] = length;
-    isNull[longValues.length + doubleValues.length + index] = false;
   }
 
-  public void assignNullString(int index) {
-    // We do not assign the value to byteValues[] because the value is never 
used on null
-    isNull[longValues.length + doubleValues.length + index] = true;
+  public void assignNullString(int keyIndex, int index) {
+    isNull[keyIndex] = true;
+    byteValues[index] = null;
+    byteStarts[index] = 0;
+    // We need some value that indicates NULL.
+    byteLengths[index] = -1;
   }
 
   public void assignDecimal(int index, HiveDecimalWritable value) {
     decimalValues[index].set(value);
-    isNull[longValues.length + doubleValues.length + byteValues.length + 
index] = false;
   }
 
-  public void assignNullDecimal(int index) {
-      isNull[longValues.length + doubleValues.length + byteValues.length + 
index] = true;
+  public void assignNullDecimal(int keyIndex, int index) {
+    isNull[keyIndex] = true;
+    decimalValues[index].set(HiveDecimal.ZERO); // assign 0 to simplify 
hashcode
   }
 
   public void assignTimestamp(int index, Timestamp value) {
     timestampValues[index] = value;
-    isNull[longValues.length + doubleValues.length + byteValues.length +
-           decimalValues.length + index] = false;
   }
 
   public void assignTimestamp(int index, TimestampColumnVector colVector, int 
elementNum) {
     colVector.timestampUpdate(timestampValues[index], elementNum);
-    isNull[longValues.length + doubleValues.length + byteValues.length +
-           decimalValues.length + index] = false;
   }
 
-  public void assignNullTimestamp(int index) {
-      isNull[longValues.length + doubleValues.length + byteValues.length +
-             decimalValues.length + index] = true;
+  public void assignNullTimestamp(int keyIndex, int index) {
+    isNull[keyIndex] = true;
+    timestampValues[index] = ZERO_TIMESTAMP; // assign 0 to simplify hashcode
   }
 
   public void assignIntervalDayTime(int index, HiveIntervalDayTime value) {
     intervalDayTimeValues[index].set(value);
-    isNull[longValues.length + doubleValues.length + byteValues.length +
-           decimalValues.length + timestampValues.length + index] = false;
   }
 
   public void assignIntervalDayTime(int index, IntervalDayTimeColumnVector 
colVector, int elementNum) {
     
intervalDayTimeValues[index].set(colVector.asScratchIntervalDayTime(elementNum));
-    isNull[longValues.length + doubleValues.length + byteValues.length +
-           decimalValues.length + timestampValues.length + index] = false;
   }
 
-  public void assignNullIntervalDayTime(int index) {
-      isNull[longValues.length + doubleValues.length + byteValues.length +
-             decimalValues.length + timestampValues.length + index] = true;
+  public void assignNullIntervalDayTime(int keyIndex, int index) {
+    isNull[keyIndex] = true;
+    intervalDayTimeValues[index] = ZERO_INTERVALDAYTIME; // assign 0 to 
simplify hashcode
   }
 
   @Override
   public String toString()
   {
-    return String.format("%d[%s] %d[%s] %d[%s] %d[%s] %d[%s] %d[%s]",
-        longValues.length, Arrays.toString(longValues),
-        doubleValues.length, Arrays.toString(doubleValues),
-        byteValues.length, Arrays.toString(byteValues),
-        decimalValues.length, Arrays.toString(decimalValues),
-        timestampValues.length, Arrays.toString(timestampValues),
-        intervalDayTimeValues.length, Arrays.toString(intervalDayTimeValues));
-  }
-
-  public boolean getIsLongNull(int i) {
-    return isNull[i];
-  }
+    StringBuilder sb = new StringBuilder();
+    boolean isFirst = true;
+    if (longValues.length > 0) {
+      isFirst = false;
+      sb.append("longs ");
+      sb.append(Arrays.toString(longValues));
+    }
+    if (doubleValues.length > 0) {
+      if (isFirst) {
+        isFirst = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("doubles ");
+      sb.append(Arrays.toString(doubleValues));
+    }
+    if (byteValues.length > 0) {
+      if (isFirst) {
+        isFirst = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("byte lengths ");
+      sb.append(Arrays.toString(byteLengths));
+    }
+    if (decimalValues.length > 0) {
+      if (isFirst) {
+        isFirst = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("decimals ");
+      sb.append(Arrays.toString(decimalValues));
+    }
+    if (timestampValues.length > 0) {
+      if (isFirst) {
+        isFirst = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("timestamps ");
+      sb.append(Arrays.toString(timestampValues));
+    }
+    if (intervalDayTimeValues.length > 0) {
+      if (isFirst) {
+        isFirst = false;
+      } else {
+        sb.append(", ");
+      }
+      sb.append("interval day times ");
+      sb.append(Arrays.toString(intervalDayTimeValues));
+    }
 
-  public boolean getIsDoubleNull(int i) {
-    return isNull[longValues.length + i];
-  }
+    if (isFirst) {
+      isFirst = false;
+    } else {
+      sb.append(", ");
+    }
+    sb.append("nulls ");
+    sb.append(Arrays.toString(isNull));
 
-  public boolean getIsBytesNull(int i) {
-    return isNull[longValues.length + doubleValues.length + i];
+    return sb.toString();
   }
 
-
   public long getLongValue(int i) {
     return longValues[i];
   }
@@ -403,35 +442,29 @@ public class VectorHashKeyWrapper extends KeyWrapper {
     return variableSize;
   }
 
-  public boolean getIsDecimalNull(int i) {
-    return isNull[longValues.length + doubleValues.length + byteValues.length 
+ i];
-  }
-
   public HiveDecimalWritable getDecimal(int i) {
     return decimalValues[i];
   }
 
-  public boolean getIsTimestampNull(int i) {
-    return isNull[longValues.length + doubleValues.length + byteValues.length +
-                  decimalValues.length + i];
-  }
-
   public Timestamp getTimestamp(int i) {
     return timestampValues[i];
   }
 
-  public boolean getIsIntervalDayTimeNull(int i) {
-    return isNull[longValues.length + doubleValues.length + byteValues.length +
-                  decimalValues.length + timestampValues.length + i];
-  }
-
   public HiveIntervalDayTime getIntervalDayTime(int i) {
     return intervalDayTimeValues[i];
   }
 
+  public void clearIsNull() {
+    Arrays.fill(isNull, false);
+  }
+
+  public boolean isNull(int keyIndex) {
+    return isNull[keyIndex];
+  }
+
   public static final class EmptyVectorHashKeyWrapper extends 
VectorHashKeyWrapper {
     private EmptyVectorHashKeyWrapper() {
-      super(0, 0, 0, 0, 0, 0);
+      super(0, 0, 0, 0, 0, 0, /* keyCount */ 0);
       // no need to override assigns - all assign ops will fail due to 0 size
     }
 
@@ -451,4 +484,3 @@ public class VectorHashKeyWrapper extends KeyWrapper {
     }
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/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 f68228c..63cdf94 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
@@ -23,6 +23,9 @@ import 
org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpressionWriter;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector.Type;
 
 /**
  * Class for handling vectorized hash map key wrappers. It evaluates the key 
columns in a
@@ -85,12 +88,168 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * @throws HiveException
    */
   public void evaluateBatch(VectorizedRowBatch batch) throws HiveException {
-    for(int i = 0; i < keyExpressions.length; ++i) {
-      keyExpressions[i].evaluate(batch);
+
+    for(int i=0;i<batch.size;++i) {
+      vectorHashKeyWrappers[i].clearIsNull();
+    }
+
+    int keyIndex;
+    int columnIndex;
+    for(int i = 0; i< longIndices.length; ++i) {
+      keyIndex = longIndices[i];
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      LongColumnVector columnVector = (LongColumnVector) 
batch.cols[columnIndex];
+      if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignLongNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+      } else if (columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignLongNoNullsNoRepeatingSelection(i, batch.size, columnVector, 
batch.selected);
+      } else if (columnVector.noNulls && columnVector.isRepeating) {
+        assignLongNoNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignLongNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
+      } else if (!columnVector.noNulls && columnVector.isRepeating) {
+        assignLongNullsRepeating(keyIndex, i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignLongNullsNoRepeatingSelection (keyIndex, i, batch.size, 
columnVector, batch.selected);
+      } else {
+        throw new HiveException (String.format(
+            "Unimplemented Long null/repeat/selected combination %b/%b/%b",
+            columnVector.noNulls, columnVector.isRepeating, 
batch.selectedInUse));
+      }
+    }
+    for(int i=0;i<doubleIndices.length; ++i) {
+      keyIndex = doubleIndices[i];
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      DoubleColumnVector columnVector = (DoubleColumnVector) 
batch.cols[columnIndex];
+      if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignDoubleNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+      } else if (columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignDoubleNoNullsNoRepeatingSelection(i, batch.size, columnVector, 
batch.selected);
+      } else if (columnVector.noNulls && columnVector.isRepeating) {
+        assignDoubleNoNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignDoubleNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
+      } else if (!columnVector.noNulls && columnVector.isRepeating) {
+        assignDoubleNullsRepeating(keyIndex, i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignDoubleNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
+      } else {
+        throw new HiveException (String.format(
+            "Unimplemented Double null/repeat/selected combination %b/%b/%b",
+            columnVector.noNulls, columnVector.isRepeating, 
batch.selectedInUse));
+      }
+    }
+    for(int i=0;i<stringIndices.length; ++i) {
+      keyIndex = stringIndices[i];
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      BytesColumnVector columnVector = (BytesColumnVector) 
batch.cols[columnIndex];
+      if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignStringNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+      } else if (columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignStringNoNullsNoRepeatingSelection(i, batch.size, columnVector, 
batch.selected);
+      } else if (columnVector.noNulls && columnVector.isRepeating) {
+        assignStringNoNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignStringNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
+      } else if (!columnVector.noNulls && columnVector.isRepeating) {
+        assignStringNullsRepeating(keyIndex, i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignStringNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
+      } else {
+        throw new HiveException (String.format(
+            "Unimplemented String null/repeat/selected combination %b/%b/%b",
+            columnVector.noNulls, columnVector.isRepeating, 
batch.selectedInUse));
+      }
+    }
+    for(int i=0;i<decimalIndices.length; ++i) {
+      keyIndex = decimalIndices[i];
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      DecimalColumnVector columnVector = (DecimalColumnVector) 
batch.cols[columnIndex];
+      if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignDecimalNoNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
+      } else if (columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignDecimalNoNullsNoRepeatingSelection(i, batch.size, columnVector, 
batch.selected);
+      } else if (columnVector.noNulls && columnVector.isRepeating) {
+        assignDecimalNoNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignDecimalNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
+      } else if (!columnVector.noNulls && columnVector.isRepeating) {
+        assignDecimalNullsRepeating(keyIndex, i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignDecimalNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
+      } else {
+        throw new HiveException (String.format(
+            "Unimplemented Decimal null/repeat/selected combination %b/%b/%b",
+            columnVector.noNulls, columnVector.isRepeating, 
batch.selectedInUse));
+      }
     }
+    for(int i=0;i<timestampIndices.length; ++i) {
+      keyIndex = timestampIndices[i];
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      TimestampColumnVector columnVector = (TimestampColumnVector) 
batch.cols[columnIndex];
+      if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignTimestampNoNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
+      } else if (columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignTimestampNoNullsNoRepeatingSelection(i, batch.size, 
columnVector, batch.selected);
+      } else if (columnVector.noNulls && columnVector.isRepeating) {
+        assignTimestampNoNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignTimestampNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
+      } else if (!columnVector.noNulls && columnVector.isRepeating) {
+        assignTimestampNullsRepeating(keyIndex, i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignTimestampNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
+      } else {
+        throw new HiveException (String.format(
+            "Unimplemented timestamp null/repeat/selected combination 
%b/%b/%b",
+            columnVector.noNulls, columnVector.isRepeating, 
batch.selectedInUse));
+      }
+    }
+    for(int i=0;i<intervalDayTimeIndices.length; ++i) {
+      keyIndex = intervalDayTimeIndices[i];
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      IntervalDayTimeColumnVector columnVector = (IntervalDayTimeColumnVector) 
batch.cols[columnIndex];
+      if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignIntervalDayTimeNoNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
+      } else if (columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignIntervalDayTimeNoNullsNoRepeatingSelection(i, batch.size, 
columnVector, batch.selected);
+      } else if (columnVector.noNulls && columnVector.isRepeating) {
+        assignIntervalDayTimeNoNullsRepeating(i, batch.size, columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
+        assignIntervalDayTimeNullsNoRepeatingNoSelection(keyIndex, i, 
batch.size, columnVector);
+      } else if (!columnVector.noNulls && columnVector.isRepeating) {
+        assignIntervalDayTimeNullsRepeating(keyIndex, i, batch.size, 
columnVector);
+      } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
+        assignIntervalDayTimeNullsNoRepeatingSelection(keyIndex, i, 
batch.size, columnVector, batch.selected);
+      } else {
+        throw new HiveException (String.format(
+            "Unimplemented intervalDayTime null/repeat/selected combination 
%b/%b/%b",
+            columnVector.noNulls, columnVector.isRepeating, 
batch.selectedInUse));
+      }
+    }
+    for(int i=0;i<batch.size;++i) {
+      vectorHashKeyWrappers[i].setHashKey();
+    }
+  }
+
+  public void evaluateBatchGroupingSets(VectorizedRowBatch batch,
+      boolean[] groupingSetsOverrideIsNulls) throws HiveException {
+
+    for(int i=0;i<batch.size;++i) {
+      vectorHashKeyWrappers[i].clearIsNull();
+    }
+    int keyIndex;
+    int columnIndex;
     for(int i = 0; i< longIndices.length; ++i) {
-      int keyIndex = longIndices[i];
-      int columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      keyIndex = longIndices[i];
+      if (groupingSetsOverrideIsNulls[keyIndex]) {
+        final int batchSize = batch.size;
+        for(int r = 0; r < batchSize; ++r) {
+          vectorHashKeyWrappers[r].assignNullLong(keyIndex, i);
+        }
+        continue;
+      }
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
       LongColumnVector columnVector = (LongColumnVector) 
batch.cols[columnIndex];
       if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
         assignLongNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
@@ -99,11 +258,11 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       } else if (columnVector.noNulls && columnVector.isRepeating) {
         assignLongNoNullsRepeating(i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
-        assignLongNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+        assignLongNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
       } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignLongNullsRepeating(i, batch.size, columnVector);
+        assignLongNullsRepeating(keyIndex, i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
-        assignLongNullsNoRepeatingSelection (i, batch.size, columnVector, 
batch.selected);
+        assignLongNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
       } else {
         throw new HiveException (String.format(
             "Unimplemented Long null/repeat/selected combination %b/%b/%b",
@@ -111,8 +270,15 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       }
     }
     for(int i=0;i<doubleIndices.length; ++i) {
-      int keyIndex = doubleIndices[i];
-      int columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      keyIndex = doubleIndices[i];
+      if (groupingSetsOverrideIsNulls[keyIndex]) {
+        final int batchSize = batch.size;
+        for(int r = 0; r < batchSize; ++r) {
+          vectorHashKeyWrappers[r].assignNullDouble(keyIndex, i);
+        }
+        continue;
+      }
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
       DoubleColumnVector columnVector = (DoubleColumnVector) 
batch.cols[columnIndex];
       if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
         assignDoubleNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
@@ -121,11 +287,11 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       } else if (columnVector.noNulls && columnVector.isRepeating) {
         assignDoubleNoNullsRepeating(i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
-        assignDoubleNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+        assignDoubleNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
       } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignDoubleNullsRepeating(i, batch.size, columnVector);
+        assignDoubleNullsRepeating(keyIndex, i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
-        assignDoubleNullsNoRepeatingSelection (i, batch.size, columnVector, 
batch.selected);
+        assignDoubleNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
       } else {
         throw new HiveException (String.format(
             "Unimplemented Double null/repeat/selected combination %b/%b/%b",
@@ -133,8 +299,15 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       }
     }
     for(int i=0;i<stringIndices.length; ++i) {
-      int keyIndex = stringIndices[i];
-      int columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      keyIndex = stringIndices[i];
+      if (groupingSetsOverrideIsNulls[keyIndex]) {
+        final int batchSize = batch.size;
+        for(int r = 0; r < batchSize; ++r) {
+          vectorHashKeyWrappers[r].assignNullString(keyIndex, i);
+        }
+        continue;
+      }
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
       BytesColumnVector columnVector = (BytesColumnVector) 
batch.cols[columnIndex];
       if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
         assignStringNoNullsNoRepeatingNoSelection(i, batch.size, columnVector);
@@ -143,11 +316,11 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       } else if (columnVector.noNulls && columnVector.isRepeating) {
         assignStringNoNullsRepeating(i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
-        assignStringNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+        assignStringNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
       } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignStringNullsRepeating(i, batch.size, columnVector);
+        assignStringNullsRepeating(keyIndex, i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
-        assignStringNullsNoRepeatingSelection (i, batch.size, columnVector, 
batch.selected);
+        assignStringNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
       } else {
         throw new HiveException (String.format(
             "Unimplemented String null/repeat/selected combination %b/%b/%b",
@@ -155,8 +328,15 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       }
     }
     for(int i=0;i<decimalIndices.length; ++i) {
-      int keyIndex = decimalIndices[i];
-      int columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      keyIndex = decimalIndices[i];
+      if (groupingSetsOverrideIsNulls[keyIndex]) {
+        final int batchSize = batch.size;
+        for(int r = 0; r < batchSize; ++r) {
+          vectorHashKeyWrappers[r].assignNullDecimal(keyIndex, i);
+        }
+        continue;
+      }
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
       DecimalColumnVector columnVector = (DecimalColumnVector) 
batch.cols[columnIndex];
       if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
         assignDecimalNoNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
@@ -165,11 +345,11 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       } else if (columnVector.noNulls && columnVector.isRepeating) {
         assignDecimalNoNullsRepeating(i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
-        assignDecimalNullsNoRepeatingNoSelection(i, batch.size, columnVector);
+        assignDecimalNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
       } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignDecimalNullsRepeating(i, batch.size, columnVector);
+        assignDecimalNullsRepeating(keyIndex, i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
-        assignDecimalNullsNoRepeatingSelection (i, batch.size, columnVector, 
batch.selected);
+        assignDecimalNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
       } else {
         throw new HiveException (String.format(
             "Unimplemented Decimal null/repeat/selected combination %b/%b/%b",
@@ -177,8 +357,15 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       }
     }
     for(int i=0;i<timestampIndices.length; ++i) {
-      int keyIndex = timestampIndices[i];
-      int columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      keyIndex = timestampIndices[i];
+      if (groupingSetsOverrideIsNulls[keyIndex]) {
+        final int batchSize = batch.size;
+        for(int r = 0; r < batchSize; ++r) {
+          vectorHashKeyWrappers[r].assignNullTimestamp(keyIndex, i);
+        }
+        continue;
+      }
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
       TimestampColumnVector columnVector = (TimestampColumnVector) 
batch.cols[columnIndex];
       if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
         assignTimestampNoNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
@@ -187,11 +374,11 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       } else if (columnVector.noNulls && columnVector.isRepeating) {
         assignTimestampNoNullsRepeating(i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
-        assignTimestampNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
+        assignTimestampNullsNoRepeatingNoSelection(keyIndex, i, batch.size, 
columnVector);
       } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignTimestampNullsRepeating(i, batch.size, columnVector);
+        assignTimestampNullsRepeating(keyIndex, i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
-        assignTimestampNullsNoRepeatingSelection (i, batch.size, columnVector, 
batch.selected);
+        assignTimestampNullsNoRepeatingSelection(keyIndex, i, batch.size, 
columnVector, batch.selected);
       } else {
         throw new HiveException (String.format(
             "Unimplemented timestamp null/repeat/selected combination 
%b/%b/%b",
@@ -199,8 +386,15 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       }
     }
     for(int i=0;i<intervalDayTimeIndices.length; ++i) {
-      int keyIndex = intervalDayTimeIndices[i];
-      int columnIndex = keyExpressions[keyIndex].getOutputColumn();
+      keyIndex = intervalDayTimeIndices[i];
+      if (groupingSetsOverrideIsNulls[keyIndex]) {
+        final int batchSize = batch.size;
+        for(int r = 0; r < batchSize; ++r) {
+          vectorHashKeyWrappers[r].assignNullIntervalDayTime(keyIndex, i);
+        }
+        continue;
+      }
+      columnIndex = keyExpressions[keyIndex].getOutputColumn();
       IntervalDayTimeColumnVector columnVector = (IntervalDayTimeColumnVector) 
batch.cols[columnIndex];
       if (columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
         assignIntervalDayTimeNoNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
@@ -209,11 +403,11 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       } else if (columnVector.noNulls && columnVector.isRepeating) {
         assignIntervalDayTimeNoNullsRepeating(i, batch.size, columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
!batch.selectedInUse) {
-        assignIntervalDayTimeNullsNoRepeatingNoSelection(i, batch.size, 
columnVector);
+        assignIntervalDayTimeNullsNoRepeatingNoSelection(keyIndex, i, 
batch.size, columnVector);
       } else if (!columnVector.noNulls && columnVector.isRepeating) {
-        assignIntervalDayTimeNullsRepeating(i, batch.size, columnVector);
+        assignIntervalDayTimeNullsRepeating(keyIndex, i, batch.size, 
columnVector);
       } else if (!columnVector.noNulls && !columnVector.isRepeating && 
batch.selectedInUse) {
-        assignIntervalDayTimeNullsNoRepeatingSelection (i, batch.size, 
columnVector, batch.selected);
+        assignIntervalDayTimeNullsNoRepeatingSelection(keyIndex, i, 
batch.size, columnVector, batch.selected);
       } else {
         throw new HiveException (String.format(
             "Unimplemented intervalDayTime null/repeat/selected combination 
%b/%b/%b",
@@ -229,14 +423,15 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for string type, possible nulls, no repeat values, batch 
selection vector.
    */
-  private void assignStringNullsNoRepeatingSelection(int index, int size,
+  private void assignStringNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
       BytesColumnVector columnVector, int[] selected) {
     for(int i=0; i<size; ++i) {
       int row = selected[i];
       if (columnVector.isNull[row]) {
-        vectorHashKeyWrappers[i].assignNullString(index);
+        vectorHashKeyWrappers[i].assignNullString(keyIndex, index);
       } else {
-        vectorHashKeyWrappers[i].assignString(index,
+        vectorHashKeyWrappers[i].assignString(
+            index,
             columnVector.vector[row],
             columnVector.start[row],
             columnVector.length[row]);
@@ -248,14 +443,15 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for double type, possible nulls, repeat values.
    */
-  private void assignStringNullsRepeating(int index, int size, 
BytesColumnVector columnVector) {
+  private void assignStringNullsRepeating(int keyIndex, int index, int size, 
BytesColumnVector columnVector) {
     if (columnVector.isNull[0]) {
       for(int i = 0; i < size; ++i) {
-        vectorHashKeyWrappers[i].assignNullString(index);
+        vectorHashKeyWrappers[i].assignNullString(keyIndex, index);
       }
     } else {
       for(int i = 0; i < size; ++i) {
-        vectorHashKeyWrappers[i].assignString(index,
+        vectorHashKeyWrappers[i].assignString(
+            index,
             columnVector.vector[0],
             columnVector.start[0],
             columnVector.length[0]);
@@ -267,13 +463,14 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for string type, possible nulls, no repeat values, no selection 
vector.
    */
-  private void assignStringNullsNoRepeatingNoSelection(int index, int size,
+  private void assignStringNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
       BytesColumnVector columnVector) {
     for(int i=0; i<size; ++i) {
       if (columnVector.isNull[i]) {
-        vectorHashKeyWrappers[i].assignNullString(index);
+        vectorHashKeyWrappers[i].assignNullString(keyIndex, index);
       } else {
-        vectorHashKeyWrappers[i].assignString(index,
+        vectorHashKeyWrappers[i].assignString(
+            index,
             columnVector.vector[i],
             columnVector.start[i],
             columnVector.length[i]);
@@ -285,9 +482,11 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for double type, no nulls, repeat values, no selection vector.
    */
-  private void assignStringNoNullsRepeating(int index, int size, 
BytesColumnVector columnVector) {
+  private void assignStringNoNullsRepeating(int index, int size,
+      BytesColumnVector columnVector) {
     for(int i = 0; i < size; ++i) {
-      vectorHashKeyWrappers[i].assignString(index,
+      vectorHashKeyWrappers[i].assignString(
+          index,
           columnVector.vector[0],
           columnVector.start[0],
           columnVector.length[0]);
@@ -302,7 +501,8 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
       BytesColumnVector columnVector, int[] selected) {
     for(int i=0; i<size; ++i) {
       int row = selected[i];
-      vectorHashKeyWrappers[i].assignString(index,
+      vectorHashKeyWrappers[i].assignString(
+          index,
           columnVector.vector[row],
           columnVector.start[row],
           columnVector.length[row]);
@@ -316,7 +516,8 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
   private void assignStringNoNullsNoRepeatingNoSelection(int index, int size,
       BytesColumnVector columnVector) {
     for(int i=0; i<size; ++i) {
-      vectorHashKeyWrappers[i].assignString(index,
+      vectorHashKeyWrappers[i].assignString(
+          index,
           columnVector.vector[i],
           columnVector.start[i],
           columnVector.length[i]);
@@ -327,14 +528,14 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for double type, possible nulls, no repeat values, batch 
selection vector.
    */
-  private void assignDoubleNullsNoRepeatingSelection(int index, int size,
+  private void assignDoubleNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
       DoubleColumnVector columnVector, int[] selected) {
     for(int i = 0; i < size; ++i) {
       int row = selected[i];
       if (!columnVector.isNull[row]) {
         vectorHashKeyWrappers[i].assignDouble(index, columnVector.vector[row]);
       } else {
-        vectorHashKeyWrappers[i].assignNullDouble(index);
+        vectorHashKeyWrappers[i].assignNullDouble(keyIndex, index);
       }
     }
   }
@@ -343,10 +544,10 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Double type, repeat null values.
    */
-  private void assignDoubleNullsRepeating(int index, int size,
+  private void assignDoubleNullsRepeating(int keyIndex, int index, int size,
       DoubleColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullDouble(index);
+      vectorHashKeyWrappers[r].assignNullDouble(keyIndex, index);
     }
   }
 
@@ -354,13 +555,13 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Double type, possible nulls, repeat values.
    */
-  private void assignDoubleNullsNoRepeatingNoSelection(int index, int size,
+  private void assignDoubleNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
       DoubleColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
       if (!columnVector.isNull[r]) {
         vectorHashKeyWrappers[r].assignDouble(index, columnVector.vector[r]);
       } else {
-        vectorHashKeyWrappers[r].assignNullDouble(index);
+        vectorHashKeyWrappers[r].assignNullDouble(keyIndex, index);
       }
     }
   }
@@ -401,14 +602,14 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for double type, possible nulls, no repeat values, batch 
selection vector.
    */
-  private void assignLongNullsNoRepeatingSelection(int index, int size,
+  private void assignLongNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
       LongColumnVector columnVector, int[] selected) {
     for(int i = 0; i < size; ++i) {
       int row = selected[i];
       if (!columnVector.isNull[row]) {
         vectorHashKeyWrappers[i].assignLong(index, columnVector.vector[row]);
       } else {
-        vectorHashKeyWrappers[i].assignNullLong(index);
+        vectorHashKeyWrappers[i].assignNullLong(keyIndex, index);
       }
     }
   }
@@ -417,10 +618,10 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for double type, repeating nulls.
    */
-  private void assignLongNullsRepeating(int index, int size,
+  private void assignLongNullsRepeating(int keyIndex, int index, int size,
       LongColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullLong(index);
+      vectorHashKeyWrappers[r].assignNullLong(keyIndex, index);
     }
   }
 
@@ -428,13 +629,13 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for double type, possible nulls, no repeat values, no selection 
vector.
    */
-  private void assignLongNullsNoRepeatingNoSelection(int index, int size,
+  private void assignLongNullsNoRepeatingNoSelection(int keyIndex, int index, 
int size,
       LongColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
       if (!columnVector.isNull[r]) {
         vectorHashKeyWrappers[r].assignLong(index, columnVector.vector[r]);
       } else {
-        vectorHashKeyWrappers[r].assignNullLong(index);
+        vectorHashKeyWrappers[r].assignNullLong(keyIndex, index);
       }
     }
   }
@@ -475,14 +676,14 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Decimal type, possible nulls, no repeat values, batch 
selection vector.
    */
-  private void assignDecimalNullsNoRepeatingSelection(int index, int size,
+  private void assignDecimalNullsNoRepeatingSelection(int keyIndex, int index, 
int size,
       DecimalColumnVector columnVector, int[] selected) {
     for(int i = 0; i < size; ++i) {
       int row = selected[i];
       if (!columnVector.isNull[row]) {
         vectorHashKeyWrappers[i].assignDecimal(index, 
columnVector.vector[row]);
       } else {
-        vectorHashKeyWrappers[i].assignNullDecimal(index);
+        vectorHashKeyWrappers[i].assignNullDecimal(keyIndex, index);
       }
     }
   }
@@ -491,10 +692,10 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Decimal type, repeat null values.
    */
-  private void assignDecimalNullsRepeating(int index, int size,
+  private void assignDecimalNullsRepeating(int keyIndex, int index, int size,
       DecimalColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullDecimal(index);
+      vectorHashKeyWrappers[r].assignNullDecimal(keyIndex, index);
     }
   }
 
@@ -502,13 +703,13 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Decimal type, possible nulls, repeat values.
    */
-  private void assignDecimalNullsNoRepeatingNoSelection(int index, int size,
+  private void assignDecimalNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
       DecimalColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
       if (!columnVector.isNull[r]) {
         vectorHashKeyWrappers[r].assignDecimal(index, columnVector.vector[r]);
       } else {
-        vectorHashKeyWrappers[r].assignNullDecimal(index);
+        vectorHashKeyWrappers[r].assignNullDecimal(keyIndex, index);
       }
     }
   }
@@ -549,14 +750,14 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Timestamp type, possible nulls, no repeat values, batch 
selection vector.
    */
-  private void assignTimestampNullsNoRepeatingSelection(int index, int size,
+  private void assignTimestampNullsNoRepeatingSelection(int keyIndex, int 
index, int size,
       TimestampColumnVector columnVector, int[] selected) {
     for(int i = 0; i < size; ++i) {
       int row = selected[i];
       if (!columnVector.isNull[row]) {
         vectorHashKeyWrappers[i].assignTimestamp(index, columnVector, row);
       } else {
-        vectorHashKeyWrappers[i].assignNullTimestamp(index);
+        vectorHashKeyWrappers[i].assignNullTimestamp(keyIndex, index);
       }
     }
   }
@@ -565,10 +766,10 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Timestamp type, repeat null values.
    */
-  private void assignTimestampNullsRepeating(int index, int size,
+  private void assignTimestampNullsRepeating(int keyIndex, int index, int size,
       TimestampColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullTimestamp(index);
+      vectorHashKeyWrappers[r].assignNullTimestamp(keyIndex, index);
     }
   }
 
@@ -576,13 +777,13 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for Timestamp type, possible nulls, repeat values.
    */
-  private void assignTimestampNullsNoRepeatingNoSelection(int index, int size,
+  private void assignTimestampNullsNoRepeatingNoSelection(int keyIndex, int 
index, int size,
       TimestampColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
       if (!columnVector.isNull[r]) {
         vectorHashKeyWrappers[r].assignTimestamp(index, columnVector, r);
       } else {
-        vectorHashKeyWrappers[r].assignNullTimestamp(index);
+        vectorHashKeyWrappers[r].assignNullTimestamp(keyIndex, index);
       }
     }
   }
@@ -623,14 +824,14 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for IntervalDayTime type, possible nulls, no repeat values, 
batch selection vector.
    */
-  private void assignIntervalDayTimeNullsNoRepeatingSelection(int index, int 
size,
+  private void assignIntervalDayTimeNullsNoRepeatingSelection(int keyIndex, 
int index, int size,
       IntervalDayTimeColumnVector columnVector, int[] selected) {
     for(int i = 0; i < size; ++i) {
       int row = selected[i];
       if (!columnVector.isNull[row]) {
         vectorHashKeyWrappers[i].assignIntervalDayTime(index, columnVector, 
row);
       } else {
-        vectorHashKeyWrappers[i].assignNullIntervalDayTime(index);
+        vectorHashKeyWrappers[i].assignNullIntervalDayTime(keyIndex, index);
       }
     }
   }
@@ -639,10 +840,10 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for IntervalDayTime type, repeat null values.
    */
-  private void assignIntervalDayTimeNullsRepeating(int index, int size,
+  private void assignIntervalDayTimeNullsRepeating(int keyIndex, int index, 
int size,
       IntervalDayTimeColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
-      vectorHashKeyWrappers[r].assignNullIntervalDayTime(index);
+      vectorHashKeyWrappers[r].assignNullIntervalDayTime(keyIndex, index);
     }
   }
 
@@ -650,13 +851,13 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
    * Helper method to assign values from a vector column into the key wrapper.
    * Optimized for IntervalDayTime type, possible nulls, repeat values.
    */
-  private void assignIntervalDayTimeNullsNoRepeatingNoSelection(int index, int 
size,
+  private void assignIntervalDayTimeNullsNoRepeatingNoSelection(int keyIndex, 
int index, int size,
       IntervalDayTimeColumnVector columnVector) {
     for(int r = 0; r < size; ++r) {
       if (!columnVector.isNull[r]) {
         vectorHashKeyWrappers[r].assignIntervalDayTime(index, columnVector, r);
       } else {
-        vectorHashKeyWrappers[r].assignNullIntervalDayTime(index);
+        vectorHashKeyWrappers[r].assignNullIntervalDayTime(keyIndex, index);
       }
     }
   }
@@ -693,13 +894,28 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
     }
   }
 
+  public static VectorHashKeyWrapperBatch 
compileKeyWrapperBatch(VectorExpression[] keyExpressions)
+      throws HiveException
+  {
+
+    final int size = keyExpressions.length;
+    ColumnVector.Type[] columnVectorTypes = new ColumnVector.Type[size];
+    for (int i = 0; i < size; i++) {
+      String typeName = 
VectorizationContext.mapTypeNameSynonyms(keyExpressions[i].getOutputType());
+      TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+      columnVectorTypes[i] = 
VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
+    }
+    return compileKeyWrapperBatch(keyExpressions, columnVectorTypes);
+  }
+
   /**
    * Prepares a VectorHashKeyWrapperBatch to work for a specific set of keys.
    * Computes the fast access lookup indices, preallocates all needed internal 
arrays.
    * This step is done only once per query, not once per batch. The 
information computed now
    * will be used to generate proper individual VectorKeyHashWrapper objects.
    */
-  public static VectorHashKeyWrapperBatch 
compileKeyWrapperBatch(VectorExpression[] keyExpressions)
+  public static VectorHashKeyWrapperBatch 
compileKeyWrapperBatch(VectorExpression[] keyExpressions,
+      ColumnVector.Type[] columnVectorTypes)
     throws HiveException {
     VectorHashKeyWrapperBatch compiledKeyWrapperBatch = new 
VectorHashKeyWrapperBatch(keyExpressions.length);
     compiledKeyWrapperBatch.keyExpressions = keyExpressions;
@@ -707,8 +923,8 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
     compiledKeyWrapperBatch.keysFixedSize = 0;
 
     // Inspect the output type of each key expression.
-    for(int i=0; i < keyExpressions.length; ++i) {
-      compiledKeyWrapperBatch.addKey(keyExpressions[i].getOutputType());
+    for(int i=0; i < columnVectorTypes.length; ++i) {
+      compiledKeyWrapperBatch.addKey(columnVectorTypes[i]);
     }
     compiledKeyWrapperBatch.finishAdding();
 
@@ -744,49 +960,54 @@ public class VectorHashKeyWrapperBatch extends 
VectorColumnSetInfo {
   }
 
   public VectorHashKeyWrapper allocateKeyWrapper() {
-    return VectorHashKeyWrapper.allocate(longIndices.length, 
doubleIndices.length,
-        stringIndices.length, decimalIndices.length, timestampIndices.length,
-        intervalDayTimeIndices.length);
+    return VectorHashKeyWrapper.allocate(
+        longIndices.length,
+        doubleIndices.length,
+        stringIndices.length,
+        decimalIndices.length,
+        timestampIndices.length,
+        intervalDayTimeIndices.length,
+        keyCount);
   }
 
   /**
    * Get the row-mode writable object value of a key from a key wrapper
    * @param keyOutputWriter
    */
-  public Object getWritableKeyValue(VectorHashKeyWrapper kw, int i,
+  public Object getWritableKeyValue(VectorHashKeyWrapper kw, int keyIndex,
       VectorExpressionWriter keyOutputWriter)
     throws HiveException {
 
-    KeyLookupHelper klh = indexLookup[i];
-    if (klh.longIndex >= 0) {
-      return kw.getIsLongNull(klh.longIndex) ? null :
-        keyOutputWriter.writeValue(kw.getLongValue(klh.longIndex));
-    } else if (klh.doubleIndex >= 0) {
-      return kw.getIsDoubleNull(klh.doubleIndex) ? null :
-          keyOutputWriter.writeValue(kw.getDoubleValue(klh.doubleIndex));
-    } else if (klh.stringIndex >= 0) {
-      return kw.getIsBytesNull(klh.stringIndex) ? null :
-          keyOutputWriter.writeValue(
-              kw.getBytes(klh.stringIndex),
-                kw.getByteStart(klh.stringIndex),
-                kw.getByteLength(klh.stringIndex));
-    } else if (klh.decimalIndex >= 0) {
-      return kw.getIsDecimalNull(klh.decimalIndex)? null :
-          keyOutputWriter.writeValue(
-                kw.getDecimal(klh.decimalIndex));
-    } else if (klh.timestampIndex >= 0) {
-      return kw.getIsTimestampNull(klh.timestampIndex)? null :
-          keyOutputWriter.writeValue(
-                kw.getTimestamp(klh.timestampIndex));
-    } else if (klh.intervalDayTimeIndex >= 0) {
-      return kw.getIsIntervalDayTimeNull(klh.intervalDayTimeIndex)? null :
-        keyOutputWriter.writeValue(
-              kw.getIntervalDayTime(klh.intervalDayTimeIndex));
-    } else {
-      throw new HiveException(String.format(
-          "Internal inconsistent KeyLookupHelper at index [%d]:%d %d %d %d %d 
%d",
-          i, klh.longIndex, klh.doubleIndex, klh.stringIndex, klh.decimalIndex,
-          klh.timestampIndex, klh.intervalDayTimeIndex));
+    if (kw.isNull(keyIndex)) {
+      return null;
+    }
+
+    ColumnVector.Type columnVectorType = columnVectorTypes[keyIndex];
+    int columnTypeSpecificIndex = columnTypeSpecificIndices[keyIndex];
+
+    switch (columnVectorType) {
+    case LONG:
+      return keyOutputWriter.writeValue(
+          kw.getLongValue(columnTypeSpecificIndex));
+    case DOUBLE:
+      return keyOutputWriter.writeValue(
+          kw.getDoubleValue(columnTypeSpecificIndex));
+    case BYTES:
+      return keyOutputWriter.writeValue(
+          kw.getBytes(columnTypeSpecificIndex),
+          kw.getByteStart(columnTypeSpecificIndex),
+          kw.getByteLength(columnTypeSpecificIndex));
+    case DECIMAL:
+      return keyOutputWriter.writeValue(
+          kw.getDecimal(columnTypeSpecificIndex));
+    case TIMESTAMP:
+      return keyOutputWriter.writeValue(
+          kw.getTimestamp(columnTypeSpecificIndex));
+    case INTERVAL_DAY_TIME:
+      return keyOutputWriter.writeValue(
+          kw.getIntervalDayTime(columnTypeSpecificIndex));
+    default:
+      throw new HiveException("Unexpected column vector type " + 
columnVectorType);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
index 848fc8e..4e05fa3 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorMapJoinOperator.java
@@ -215,6 +215,9 @@ public class VectorMapJoinOperator extends 
VectorMapJoinBaseOperator {
       }
     }
 
+    for (VectorExpression ve : keyExpressions) {
+      ve.evaluate(inBatch);
+    }
     keyWrapperBatch.evaluateBatch(inBatch);
     keyValues = keyWrapperBatch.getVectorHashKeyWrappers();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
index ac3363e..f8c4223 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSMBMapJoinOperator.java
@@ -257,6 +257,9 @@ public class VectorSMBMapJoinOperator extends 
SMBMapJoinOperator implements Vect
         }
       }
 
+      for (VectorExpression ve : keyExpressions) {
+        ve.evaluate(inBatch);
+      }
       keyWrapperBatch.evaluateBatch(inBatch);
       keyValues = keyWrapperBatch.getVectorHashKeyWrappers();
 

http://git-wip-us.apache.org/repos/asf/hive/blob/47ef02e4/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
----------------------------------------------------------------------
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java 
b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index da285f5..2b3eab4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -1901,14 +1901,6 @@ public class Vectorizer implements PhysicalPlanResolver {
   private boolean validateGroupByOperator(GroupByOperator op, boolean 
isReduce, boolean isTezOrSpark) {
     GroupByDesc desc = op.getConf();
 
-    if (desc.isGroupingSetsPresent()) {
-      setOperatorIssue("Grouping sets not supported");
-      return false;
-    }
-    if (desc.pruneGroupingSetId()) {
-      setOperatorIssue("Pruning grouping set id not supported");
-      return false;
-    }
     if (desc.getMode() != GroupByDesc.Mode.HASH && desc.isDistinct()) {
       setOperatorIssue("DISTINCT not supported");
       return false;
@@ -2018,6 +2010,11 @@ public class Vectorizer implements PhysicalPlanResolver {
 
     ProcessingMode processingMode =
         
VectorGroupByDesc.groupByDescModeToVectorProcessingMode(desc.getMode(), 
hasKeys);
+    if (desc.isGroupingSetsPresent() &&
+        (processingMode != ProcessingMode.HASH && processingMode != 
ProcessingMode.STREAMING)) {
+      LOG.info("Vectorized GROUPING SETS only expected for HASH and STREAMING 
processing modes");
+      return false;
+    }
 
     Pair<Boolean,Boolean> retPair =
         validateAggregationDescs(desc.getAggregators(), processingMode, 
hasKeys);
@@ -2222,6 +2219,12 @@ public class Vectorizer implements PhysicalPlanResolver {
       setExpressionIssue("Aggregation Function", "UDF " + udfName + " not 
supported");
       return new Pair<Boolean,Boolean>(false, false);
     }
+    /*
+    if (aggDesc.getDistinct()) {
+      setExpressionIssue("Aggregation Function", "DISTINCT not supported");
+      return new Pair<Boolean,Boolean>(false, false);
+    }
+    */
     if (aggDesc.getParameters() != null && 
!validateExprNodeDesc(aggDesc.getParameters(), "Aggregation Function UDF " + 
udfName + " parameter")) {
       return new Pair<Boolean,Boolean>(false, false);
     }

Reply via email to