You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2014/12/19 03:58:35 UTC

spark git commit: [SPARK-2663] [SQL] Support the Grouping Set

Repository: spark
Updated Branches:
  refs/heads/master 9804a759b -> f728e0fe7


[SPARK-2663] [SQL] Support the Grouping Set

Add support for `GROUPING SETS`, `ROLLUP`, `CUBE` and the the virtual column `GROUPING__ID`.

More details on how to use the `GROUPING SETS" can be found at: https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation,+Cube,+Grouping+and+Rollup
https://issues.apache.org/jira/secure/attachment/12676811/grouping_set.pdf

The generic idea of the implementations are :
1 Replace the `ROLLUP`, `CUBE` with `GROUPING SETS`
2 Explode each of the input row, and then feed them to `Aggregate`
  * Each grouping set are represented as the bit mask for the `GroupBy Expression List`, for each bit, `1` means the expression is selected, otherwise `0` (left is the lower bit, and right is the higher bit in the `GroupBy Expression List`)
  * Several of projections are constructed according to the grouping sets, and within each projection(Seq[Expression), we replace those expressions with `Literal(null)` if it's not selected in the grouping set (based on the bit mask)
  * Output Schema of `Explode` is `child.output :+ grouping__id`
  * GroupBy Expressions of `Aggregate` is `GroupBy Expression List :+ grouping__id`
  * Keep the `Aggregation expressions` the same for the `Aggregate`

The expressions substitutions happen in Logic Plan analyzing, so we will benefit from the Logical Plan optimization (e.g. expression constant folding, and map side aggregation etc.), Only an `Explosive` operator added for Physical Plan, which will explode the rows according the pre-set projections.

A known issue will be done in the follow up PR:
* Optimization `ColumnPruning` is not supported yet for `Explosive` node.

Author: Cheng Hao <ha...@intel.com>

Closes #1567 from chenghao-intel/grouping_sets and squashes the following commits:

fe65fcc [Cheng Hao] Remove the extra space
3547056 [Cheng Hao] Add more doc and Simplify the Expand
a7c869d [Cheng Hao] update code as feedbacks
d23c672 [Cheng Hao] Add GroupingExpression to replace the Seq[Expression]
414b165 [Cheng Hao] revert the unnecessary changes
ec276c6 [Cheng Hao] Support Rollup/Cube/GroupingSets


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

Branch: refs/heads/master
Commit: f728e0fe7e860fe6dd3437e248472a67a2d435f8
Parents: 9804a75
Author: Cheng Hao <ha...@intel.com>
Authored: Thu Dec 18 18:58:29 2014 -0800
Committer: Michael Armbrust <mi...@databricks.com>
Committed: Thu Dec 18 18:58:29 2014 -0800

----------------------------------------------------------------------
 .../spark/sql/catalyst/analysis/Analyzer.scala  | 95 ++++++++++++++++++++
 .../sql/catalyst/expressions/Expression.scala   | 13 ++-
 .../catalyst/expressions/namedExpressions.scala |  5 ++
 .../catalyst/plans/logical/basicOperators.scala | 83 +++++++++++++++++
 .../org/apache/spark/sql/execution/Expand.scala | 79 ++++++++++++++++
 .../spark/sql/execution/SparkStrategies.scala   |  2 +
 .../hive/execution/HiveCompatibilitySuite.scala |  7 ++
 .../org/apache/spark/sql/hive/HiveQl.scala      | 79 +++++++++++++---
 ...uping_id1-0-caf1c5fd299fdbdb655234d01d44caf2 |  0
 ...uping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b |  0
 ...uping_id1-1-c0f14def6a135cc50cba364e810ce28e |  0
 ...uping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa | 18 ++++
 ...uping_id1-3-d113d984a30ad7b98c50a46158605a51 | 12 +++
 ...uping_id1-4-903a499840102e2cb722dd8b83820391 | 18 ++++
 ...uping_id2-0-20539c642c514a590caca2f11395007e |  0
 ...uping_id2-1-d1f3215e349f056c8de60b87a6a9855e |  0
 ...uping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e |  0
 ...uping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 |  1 +
 ...uping_id2-3-b3cb95405b1200603f40aaab24be7586 | 11 +++
 ...uping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 |  3 +
 ...uping_id2-5-7dd97bda7e2a900dfc509a6133620b82 | 53 +++++++++++
 ...uping_id2-6-a7dc16cb82c595b18d4258a38a304b1e |  1 +
 ...uping_id2-7-b3cb95405b1200603f40aaab24be7586 | 11 +++
 ...uping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 |  3 +
 ...uping_id2-9-7dd97bda7e2a900dfc509a6133620b82 | 53 +++++++++++
 ...ing_sets1-0-91128722f50ec00b51e0bf6fe5695cd1 |  0
 ...ing_sets1-1-3673d61d0944adeba77438d882839de4 |  0
 ...ing_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5 |  0
 ...ing_sets1-2-5c14fabebc5b4c526c459a6e867ec61a |  6 ++
 ...ing_sets1-3-9f501f293fe180bf6322e93d8dea025a | 15 ++++
 ...ing_sets1-4-505ac6611b332d6cf4a364739075d49c | 15 ++++
 ...ing_sets1-5-a33cc72bf5adee428eea079847034b62 | 11 +++
 ...ing_sets1-6-5f0c2e1d5489c867261e575625349542 | 14 +++
 ...ing_sets1-7-9b3bda02733476012e2cda434d936423 |  5 ++
 ...ing_sets1-8-1273ad6760f8c3ddad07819362dcc324 |  5 ++
 ...ing_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 |  1 +
 ...ing_sets2-1-91128722f50ec00b51e0bf6fe5695cd1 |  0
 ...ng_sets2-10-f39e49610430c91e5af3876d15fbdfe3 | 15 ++++
 ...ing_sets2-2-3673d61d0944adeba77438d882839de4 |  0
 ...ing_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5 |  0
 ...ing_sets2-3-59adefab34d80e8e185b2ad03877d381 |  0
 ...ing_sets2-4-9f501f293fe180bf6322e93d8dea025a | 15 ++++
 ...ing_sets2-5-1163c486fd7e2c4346805fb035e2f268 |  0
 ...ing_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc | 15 ++++
 ...ing_sets2-7-39db88427f92cb770b6daa38610c04e7 |  0
 ...ing_sets2-8-e464ec3d5461bda47eac3d1ef8617786 |  0
 ...ing_sets2-9-74126b100714164d13cbb3bff436c2ff |  0
 ...ing_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc |  0
 ...ing_sets3-1-38373b67d392924967a4695689d2164e |  0
 ...ing_sets3-1-e8964b2aaeb388064c9fdac5ec687824 |  0
 ...ping_sets3-2-c1942a377b1a440d4ed3dd05fed445d |  0
 ...ing_sets3-2-d95cf9dfae402d369f338b8516845e02 |  0
 ...ing_sets3-3-b89ea2173180c8ae423d856f943e061f |  1 +
 ...ing_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 |  1 +
 ...ing_sets3-5-6623f95d90d929a6d2c8171a0698d4fd |  0
 ...ing_sets3-6-aec59088408cc57248851d3ce04e2eef | 16 ++++
 ...ing_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 |  1 +
 ...ing_sets3-8-f32c4a191759237733a10cd721b49966 |  0
 ...ing_sets3-9-aec59088408cc57248851d3ce04e2eef | 16 ++++
 ...ing_sets4-0-d8ae5a3e613dd2dda392995b90d47565 |  1 +
 ...ing_sets4-1-6ad4e855adb49babfa3ae6abac190be3 |  1 +
 ...ing_sets4-2-1789808269c8bd0f6259227f07da1a6a |  0
 ...ing_sets4-3-3673d61d0944adeba77438d882839de4 |  0
 ...ing_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5 |  0
 ...ing_sets4-4-b51d813d0c5a410d8cf9765d85005a01 |  0
 ...ing_sets4-5-9be98faf8588a3c8e7436f14c638e438 | 13 +++
 ...ing_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 |  1 +
 ...ing_sets4-7-98f7522ce136cdffb0c14163e613c250 |  0
 ...ing_sets4-8-9be98faf8588a3c8e7436f14c638e438 | 13 +++
 ...ing_sets5-0-d8ae5a3e613dd2dda392995b90d47565 |  1 +
 ...ing_sets5-1-6ad4e855adb49babfa3ae6abac190be3 |  1 +
 ...ing_sets5-2-1789808269c8bd0f6259227f07da1a6a |  0
 ...ing_sets5-3-3673d61d0944adeba77438d882839de4 |  0
 ...ing_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5 |  0
 ...ing_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9 |  0
 ...ing_sets5-5-af3569757b9f52fb9b1ead33130e1b4f | 15 ++++
 ...ing_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 |  1 +
 ...ing_sets5-7-6544a382d851f916616c4386fdcf0ed8 |  0
 ...ing_sets5-8-af3569757b9f52fb9b1ead33130e1b4f | 15 ++++
 79 files changed, 750 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 3705fcc..1c4088b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -17,11 +17,13 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
+import org.apache.spark.util.collection.OpenHashSet
 import org.apache.spark.sql.catalyst.errors.TreeNodeException
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
 import org.apache.spark.sql.catalyst.types.StructType
+import org.apache.spark.sql.catalyst.types.IntegerType
 
 /**
  * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing
@@ -56,6 +58,7 @@ class Analyzer(catalog: Catalog,
     Batch("Resolution", fixedPoint,
       ResolveReferences ::
       ResolveRelations ::
+      ResolveGroupingAnalytics ::
       ResolveSortReferences ::
       NewRelationInstances ::
       ImplicitGenerate ::
@@ -102,6 +105,93 @@ class Analyzer(catalog: Catalog,
     }
   }
 
+  object ResolveGroupingAnalytics extends Rule[LogicalPlan] {
+    /**
+     * Extract attribute set according to the grouping id
+     * @param bitmask bitmask to represent the selected of the attribute sequence
+     * @param exprs the attributes in sequence
+     * @return the attributes of non selected specified via bitmask (with the bit set to 1)
+     */
+    private def buildNonSelectExprSet(bitmask: Int, exprs: Seq[Expression])
+    : OpenHashSet[Expression] = {
+      val set = new OpenHashSet[Expression](2)
+
+      var bit = exprs.length - 1
+      while (bit >= 0) {
+        if (((bitmask >> bit) & 1) == 0) set.add(exprs(bit))
+        bit -= 1
+      }
+
+      set
+    }
+
+    /*
+     *  GROUP BY a, b, c, WITH ROLLUP
+     *  is equivalent to
+     *  GROUP BY a, b, c GROUPING SETS ( (a, b, c), (a, b), (a), ( )).
+     *  Group Count: N + 1 (N is the number of group expression)
+     *
+     *  We need to get all of its subsets for the rule described above, the subset is
+     *  represented as the bit masks.
+     */
+    def bitmasks(r: Rollup): Seq[Int] = {
+      Seq.tabulate(r.groupByExprs.length + 1)(idx => {(1 << idx) - 1})
+    }
+
+    /*
+     *  GROUP BY a, b, c, WITH CUBE
+     *  is equivalent to
+     *  GROUP BY a, b, c GROUPING SETS ( (a, b, c), (a, b), (b, c), (a, c), (a), (b), (c), ( ) ).
+     *  Group Count: 2^N (N is the number of group expression)
+     *
+     *  We need to get all of its sub sets for a given GROUPBY expressions, the subset is
+     *  represented as the bit masks.
+     */
+    def bitmasks(c: Cube): Seq[Int] = {
+      Seq.tabulate(1 << c.groupByExprs.length)(i => i)
+    }
+
+    /**
+     * Create an array of Projections for the child projection, and replace the projections'
+     * expressions which equal GroupBy expressions with Literal(null), if those expressions
+     * are not set for this grouping set (according to the bit mask).
+     */
+    private[this] def expand(g: GroupingSets): Seq[GroupExpression] = {
+      val result = new scala.collection.mutable.ArrayBuffer[GroupExpression]
+
+      g.bitmasks.foreach { bitmask =>
+        // get the non selected grouping attributes according to the bit mask
+        val nonSelectedGroupExprSet = buildNonSelectExprSet(bitmask, g.groupByExprs)
+
+        val substitution = (g.child.output :+ g.gid).map(expr => expr transformDown {
+          case x: Expression if nonSelectedGroupExprSet.contains(x) =>
+            // if the input attribute in the Invalid Grouping Expression set of for this group
+            // replace it with constant null
+            Literal(null, expr.dataType)
+          case x if x == g.gid =>
+            // replace the groupingId with concrete value (the bit mask)
+            Literal(bitmask, IntegerType)
+        })
+
+        result += GroupExpression(substitution)
+      }
+
+      result.toSeq
+    }
+
+    def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+      case a: Cube if a.resolved =>
+        GroupingSets(bitmasks(a), a.groupByExprs, a.child, a.aggregations, a.gid)
+      case a: Rollup if a.resolved =>
+        GroupingSets(bitmasks(a), a.groupByExprs, a.child, a.aggregations, a.gid)
+      case x: GroupingSets if x.resolved =>
+        Aggregate(
+          x.groupByExprs :+ x.gid,
+          x.aggregations,
+          Expand(expand(x), x.child.output :+ x.gid, x.child))
+    }
+  }
+
   /**
    * Checks for non-aggregated attributes with aggregation
    */
@@ -183,6 +273,11 @@ class Analyzer(catalog: Catalog,
       case q: LogicalPlan =>
         logTrace(s"Attempting to resolve ${q.simpleString}")
         q transformExpressions {
+          case u @ UnresolvedAttribute(name)
+              if resolver(name, VirtualColumn.groupingIdName) &&
+                q.isInstanceOf[GroupingAnalytics] =>
+              // Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics
+            q.asInstanceOf[GroupingAnalytics].gid
           case u @ UnresolvedAttribute(name) =>
             // Leave unchanged if resolution fails.  Hopefully will be resolved next round.
             val result = q.resolveChildren(name, resolver).getOrElse(u)

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index bc45881..ac5b02c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -284,6 +284,17 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression]
 
 abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] {
   self: Product =>
+}
 
-
+// TODO Semantically we probably not need GroupExpression
+// All we need is holding the Seq[Expression], and ONLY used in doing the
+// expressions transformation correctly. Probably will be removed since it's
+// not like a real expressions.
+case class GroupExpression(children: Seq[Expression]) extends Expression {
+  self: Product =>
+  type EvaluatedType = Seq[Any]
+  override def eval(input: Row): EvaluatedType = ???
+  override def nullable = false
+  override def foldable = false
+  override def dataType = ???
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index 7634d39..a3c300b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -187,3 +187,8 @@ case class AttributeReference(
 
   override def toString: String = s"$name#${exprId.id}$typeSuffix"
 }
+
+object VirtualColumn {
+  val groupingIdName = "grouping__id"
+  def newGroupingId = AttributeReference(groupingIdName, IntegerType, false)()
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 64b8d45..a9282b9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -143,6 +143,89 @@ case class Aggregate(
   override def output = aggregateExpressions.map(_.toAttribute)
 }
 
+/**
+ * Apply the all of the GroupExpressions to every input row, hence we will get
+ * multiple output rows for a input row.
+ * @param projections The group of expressions, all of the group expressions should
+ *                    output the same schema specified by the parameter `output`
+ * @param output      The output Schema
+ * @param child       Child operator
+ */
+case class Expand(
+    projections: Seq[GroupExpression],
+    output: Seq[Attribute],
+    child: LogicalPlan) extends UnaryNode
+
+trait GroupingAnalytics extends UnaryNode {
+  self: Product =>
+  def gid: AttributeReference
+  def groupByExprs: Seq[Expression]
+  def aggregations: Seq[NamedExpression]
+
+  override def output = aggregations.map(_.toAttribute)
+}
+
+/**
+ * A GROUP BY clause with GROUPING SETS can generate a result set equivalent
+ * to generated by a UNION ALL of multiple simple GROUP BY clauses.
+ *
+ * We will transform GROUPING SETS into logical plan Aggregate(.., Expand) in Analyzer
+ * @param bitmasks     A list of bitmasks, each of the bitmask indicates the selected
+ *                     GroupBy expressions
+ * @param groupByExprs The Group By expressions candidates, take effective only if the
+ *                     associated bit in the bitmask set to 1.
+ * @param child        Child operator
+ * @param aggregations The Aggregation expressions, those non selected group by expressions
+ *                     will be considered as constant null if it appears in the expressions
+ * @param gid          The attribute represents the virtual column GROUPING__ID, and it's also
+ *                     the bitmask indicates the selected GroupBy Expressions for each
+ *                     aggregating output row.
+ *                     The associated output will be one of the value in `bitmasks`
+ */
+case class GroupingSets(
+    bitmasks: Seq[Int],
+    groupByExprs: Seq[Expression],
+    child: LogicalPlan,
+    aggregations: Seq[NamedExpression],
+    gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics
+
+/**
+ * Cube is a syntactic sugar for GROUPING SETS, and will be transformed to GroupingSets,
+ * and eventually will be transformed to Aggregate(.., Expand) in Analyzer
+ *
+ * @param groupByExprs The Group By expressions candidates.
+ * @param child        Child operator
+ * @param aggregations The Aggregation expressions, those non selected group by expressions
+ *                     will be considered as constant null if it appears in the expressions
+ * @param gid          The attribute represents the virtual column GROUPING__ID, and it's also
+ *                     the bitmask indicates the selected GroupBy Expressions for each
+ *                     aggregating output row.
+ */
+case class Cube(
+    groupByExprs: Seq[Expression],
+    child: LogicalPlan,
+    aggregations: Seq[NamedExpression],
+    gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics
+
+/**
+ * Rollup is a syntactic sugar for GROUPING SETS, and will be transformed to GroupingSets,
+ * and eventually will be transformed to Aggregate(.., Expand) in Analyzer
+ *
+ * @param groupByExprs The Group By expressions candidates, take effective only if the
+ *                     associated bit in the bitmask set to 1.
+ * @param child        Child operator
+ * @param aggregations The Aggregation expressions, those non selected group by expressions
+ *                     will be considered as constant null if it appears in the expressions
+ * @param gid          The attribute represents the virtual column GROUPING__ID, and it's also
+ *                     the bitmask indicates the selected GroupBy Expressions for each
+ *                     aggregating output row.
+ */
+case class Rollup(
+    groupByExprs: Seq[Expression],
+    child: LogicalPlan,
+    aggregations: Seq[NamedExpression],
+    gid: AttributeReference = VirtualColumn.newGroupingId) extends GroupingAnalytics
+
 case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode {
   override def output = child.output
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
new file mode 100644
index 0000000..9517242
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Expand.scala
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.errors._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.physical.{UnknownPartitioning, Partitioning}
+
+/**
+ * Apply the all of the GroupExpressions to every input row, hence we will get
+ * multiple output rows for a input row.
+ * @param projections The group of expressions, all of the group expressions should
+ *                    output the same schema specified bye the parameter `output`
+ * @param output      The output Schema
+ * @param child       Child operator
+ */
+@DeveloperApi
+case class Expand(
+    projections: Seq[GroupExpression],
+    output: Seq[Attribute],
+    child: SparkPlan)
+  extends UnaryNode {
+
+  // The GroupExpressions can output data with arbitrary partitioning, so set it
+  // as UNKNOWN partitioning
+  override def outputPartitioning: Partitioning = UnknownPartitioning(0)
+
+  override def execute() = attachTree(this, "execute") {
+    child.execute().mapPartitions { iter =>
+      // TODO Move out projection objects creation and transfer to
+      // workers via closure. However we can't assume the Projection
+      // is serializable because of the code gen, so we have to
+      // create the projections within each of the partition processing.
+      val groups = projections.map(ee => newProjection(ee.children, child.output)).toArray
+
+      new Iterator[Row] {
+        private[this] var result: Row = _
+        private[this] var idx = -1  // -1 means the initial state
+        private[this] var input: Row = _
+
+        override final def hasNext = (-1 < idx && idx < groups.length) || iter.hasNext
+
+        override final def next(): Row = {
+          if (idx <= 0) {
+            // in the initial (-1) or beginning(0) of a new input row, fetch the next input tuple
+            input = iter.next()
+            idx = 0
+          }
+
+          result = groups(idx)(input)
+          idx += 1
+
+          if (idx == groups.length && iter.hasNext) {
+            idx = 0
+          }
+
+          result
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 1225d18..6e04f26 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -270,6 +270,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
         execution.Project(projectList, planLater(child)) :: Nil
       case logical.Filter(condition, child) =>
         execution.Filter(condition, planLater(child)) :: Nil
+      case logical.Expand(projections, output, child) =>
+        execution.Expand(projections, output, planLater(child)) :: Nil
       case logical.Aggregate(group, agg, child) =>
         execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil
       case logical.Sample(fraction, withReplacement, seed, child) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 4302da5..1e44dd2 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -403,6 +403,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
     "groupby11",
     "groupby12",
     "groupby1_limit",
+    "groupby_grouping_id1",
+    "groupby_grouping_id2",
+    "groupby_grouping_sets1",
+    "groupby_grouping_sets2",
+    "groupby_grouping_sets3",
+    "groupby_grouping_sets4",
+    "groupby_grouping_sets5",
     "groupby1_map",
     "groupby1_map_nomap",
     "groupby1_map_skew",

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index cd4e5a2..5939276 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -393,6 +393,42 @@ private[hive] object HiveQl {
     (db, tableName)
   }
 
+  /**
+   * SELECT MAX(value) FROM src GROUP BY k1, k2, k3 GROUPING SETS((k1, k2), (k2)) 
+   * is equivalent to 
+   * SELECT MAX(value) FROM src GROUP BY k1, k2 UNION SELECT MAX(value) FROM src GROUP BY k2
+   * Check the following link for details.
+   * 
+https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C+Grouping+and+Rollup
+   *
+   * The bitmask denotes the grouping expressions validity for a grouping set,
+   * the bitmask also be called as grouping id (`GROUPING__ID`, the virtual column in Hive)
+   * e.g. In superset (k1, k2, k3), (bit 0: k1, bit 1: k2, and bit 2: k3), the grouping id of 
+   * GROUPING SETS (k1, k2) and (k2) should be 3 and 2 respectively.
+   */
+  protected def extractGroupingSet(children: Seq[ASTNode]): (Seq[Expression], Seq[Int]) = {
+    val (keyASTs, setASTs) = children.partition( n => n match {
+        case Token("TOK_GROUPING_SETS_EXPRESSION", children) => false // grouping sets
+        case _ => true // grouping keys
+      })
+
+    val keys = keyASTs.map(nodeToExpr).toSeq
+    val keyMap = keyASTs.map(_.toStringTree).zipWithIndex.toMap
+
+    val bitmasks: Seq[Int] = setASTs.map(set => set match {
+      case Token("TOK_GROUPING_SETS_EXPRESSION", null) => 0
+      case Token("TOK_GROUPING_SETS_EXPRESSION", children) => 
+        children.foldLeft(0)((bitmap, col) => {
+          val colString = col.asInstanceOf[ASTNode].toStringTree()
+          require(keyMap.contains(colString), s"$colString doens't show up in the GROUP BY list")
+          bitmap | 1 << keyMap(colString)
+        })
+      case _ => sys.error("Expect GROUPING SETS clause")
+    })
+
+    (keys, bitmasks)
+  }
+
   protected def nodeToPlan(node: Node): LogicalPlan = node match {
     // Special drop table that also uncaches.
     case Token("TOK_DROPTABLE",
@@ -520,6 +556,9 @@ private[hive] object HiveQl {
             selectDistinctClause ::
             whereClause ::
             groupByClause ::
+            rollupGroupByClause ::
+            cubeGroupByClause ::
+            groupingSetsClause ::
             orderByClause ::
             havingClause ::
             sortByClause ::
@@ -535,6 +574,9 @@ private[hive] object HiveQl {
               "TOK_SELECTDI",
               "TOK_WHERE",
               "TOK_GROUPBY",
+              "TOK_ROLLUP_GROUPBY",
+              "TOK_CUBE_GROUPBY",
+              "TOK_GROUPING_SETS",
               "TOK_ORDERBY",
               "TOK_HAVING",
               "TOK_SORTBY",
@@ -603,16 +645,33 @@ private[hive] object HiveQl {
 
         // The projection of the query can either be a normal projection, an aggregation
         // (if there is a group by) or a script transformation.
-        val withProject = transformation.getOrElse {
-          // Not a transformation so must be either project or aggregation.
-          val selectExpressions = nameExpressions(select.getChildren.flatMap(selExprNodeToExpr))
-
-          groupByClause match {
-            case Some(groupBy) =>
-              Aggregate(groupBy.getChildren.map(nodeToExpr), selectExpressions, withLateralView)
-            case None =>
-              Project(selectExpressions, withLateralView)
-          }
+        val withProject: LogicalPlan = transformation.getOrElse {
+          val selectExpressions = 
+            nameExpressions(select.getChildren.flatMap(selExprNodeToExpr).toSeq)
+          Seq(
+            groupByClause.map(e => e match {
+              case Token("TOK_GROUPBY", children) =>
+                // Not a transformation so must be either project or aggregation.
+                Aggregate(children.map(nodeToExpr), selectExpressions, withLateralView)
+              case _ => sys.error("Expect GROUP BY")
+            }),
+            groupingSetsClause.map(e => e match {
+              case Token("TOK_GROUPING_SETS", children) =>
+                val(groupByExprs, masks) = extractGroupingSet(children)
+                GroupingSets(masks, groupByExprs, withLateralView, selectExpressions)
+              case _ => sys.error("Expect GROUPING SETS")
+            }),
+            rollupGroupByClause.map(e => e match {
+              case Token("TOK_ROLLUP_GROUPBY", children) =>
+                Rollup(children.map(nodeToExpr), withLateralView, selectExpressions)
+              case _ => sys.error("Expect WITH ROLLUP")
+            }),
+            cubeGroupByClause.map(e => e match {
+              case Token("TOK_CUBE_GROUPBY", children) =>
+                Cube(children.map(nodeToExpr), withLateralView, selectExpressions)
+              case _ => sys.error("Expect WITH CUBE")
+            }), 
+            Some(Project(selectExpressions, withLateralView))).flatten.head
         }
 
         val withDistinct =

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-0-caf1c5fd299fdbdb655234d01d44caf2
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-b76bf9f6c92f83c9a5f351f8460d1e3b
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e b/sql/hive/src/test/resources/golden/groupby_grouping_id1-1-c0f14def6a135cc50cba364e810ce28e
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa b/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa
new file mode 100644
index 0000000..76280c6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-2-3dfbff77a9b56d44312814d3fb0d07fa
@@ -0,0 +1,18 @@
+NULL	NULL	0
+NULL	11	2
+NULL	12	2
+NULL	13	2
+NULL	17	2
+NULL	18	2
+NULL	28	2
+1	NULL	1
+1	11	3
+2	NULL	1
+2	12	3
+3	NULL	1
+3	13	3
+7	NULL	1
+7	17	3
+8	NULL	1
+8	18	3
+8	28	3

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51
new file mode 100644
index 0000000..b18af4e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-3-d113d984a30ad7b98c50a46158605a51
@@ -0,0 +1,12 @@
+0	NULL	NULL
+1	1	NULL
+3	1	11
+1	2	NULL
+3	2	12
+1	3	NULL
+3	3	13
+1	7	NULL
+3	7	17
+1	8	NULL
+3	8	18
+3	8	28

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391 b/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391
new file mode 100644
index 0000000..5a7ac19
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id1-4-903a499840102e2cb722dd8b83820391
@@ -0,0 +1,18 @@
+NULL	NULL	0	0
+NULL	11	2	2
+NULL	12	2	2
+NULL	13	2	2
+NULL	17	2	2
+NULL	18	2	2
+NULL	28	2	2
+1	NULL	1	1
+1	11	3	3
+2	NULL	1	1
+2	12	3	3
+3	NULL	1	1
+3	13	3	3
+7	NULL	1	1
+7	17	3	3
+8	NULL	1	1
+8	18	3	3
+8	28	3	3

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-0-20539c642c514a590caca2f11395007e
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d1f3215e349f056c8de60b87a6a9855e
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-1-d6fb6c164cda6a13a71290dbf95fcc6e
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-2-a00d1791b7fa7ac5a0505d95c3d12257
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586
new file mode 100644
index 0000000..66ac2d6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-3-b3cb95405b1200603f40aaab24be7586
@@ -0,0 +1,11 @@
+NULL	NULL	0	6
+1	NULL	1	2
+1	NULL	3	1
+1	1	3	1
+2	NULL	1	1
+2	2	3	1
+3	NULL	1	2
+3	NULL	3	1
+3	3	3	1
+4	NULL	1	1
+4	5	3	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74
new file mode 100644
index 0000000..8c1e963
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-4-9bdcf67baa5d7cf70cd2eb1a3ec6de74
@@ -0,0 +1,3 @@
+0	1
+1	4
+3	6

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82
new file mode 100644
index 0000000..42fc229
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-5-7dd97bda7e2a900dfc509a6133620b82
@@ -0,0 +1,53 @@
+0	0
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-6-a7dc16cb82c595b18d4258a38a304b1e
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586
new file mode 100644
index 0000000..66ac2d6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-7-b3cb95405b1200603f40aaab24be7586
@@ -0,0 +1,11 @@
+NULL	NULL	0	6
+1	NULL	1	2
+1	NULL	3	1
+1	1	3	1
+2	NULL	1	1
+2	2	3	1
+3	NULL	1	2
+3	NULL	3	1
+3	3	3	1
+4	NULL	1	1
+4	5	3	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74
new file mode 100644
index 0000000..8c1e963
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-8-9bdcf67baa5d7cf70cd2eb1a3ec6de74
@@ -0,0 +1,3 @@
+0	1
+1	4
+3	6

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82 b/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82
new file mode 100644
index 0000000..42fc229
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_id2-9-7dd97bda7e2a900dfc509a6133620b82
@@ -0,0 +1,53 @@
+0	0
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3
+1	1
+1	1
+1	1
+1	1
+3	3
+3	3
+3	3
+3	3
+3	3
+3	3

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-0-91128722f50ec00b51e0bf6fe5695cd1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-1-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a
new file mode 100644
index 0000000..7967c04
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-2-5c14fabebc5b4c526c459a6e867ec61a
@@ -0,0 +1,6 @@
+8	1	1
+5	2	2
+1	1	3
+2	2	4
+2	3	5
+3	2	8

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a
new file mode 100644
index 0000000..da4a754
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-3-9f501f293fe180bf6322e93d8dea025a
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c
new file mode 100644
index 0000000..da4a754
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-4-505ac6611b332d6cf4a364739075d49c
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62
new file mode 100644
index 0000000..2d1b73d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-5-a33cc72bf5adee428eea079847034b62
@@ -0,0 +1,11 @@
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542
new file mode 100644
index 0000000..fe6c4d0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-6-5f0c2e1d5489c867261e575625349542
@@ -0,0 +1,14 @@
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+NULL
+1
+2
+3
+5
+8

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423
new file mode 100644
index 0000000..24d5fc2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-7-9b3bda02733476012e2cda434d936423
@@ -0,0 +1,5 @@
+1
+2
+3
+5
+8

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324 b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324
new file mode 100644
index 0000000..8ba0d6d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets1-8-1273ad6760f8c3ddad07819362dcc324
@@ -0,0 +1,5 @@
+2.0	1
+4.0	1
+5.0	2
+7.0	1
+9.0	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-0-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-1-91128722f50ec00b51e0bf6fe5695cd1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3
new file mode 100644
index 0000000..cabc9bb
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-10-f39e49610430c91e5af3876d15fbdfe3
@@ -0,0 +1,15 @@
+NULL	NULL	46
+NULL	1	8
+NULL	2	28
+NULL	3	10
+1	NULL	6
+1	1	6
+2	NULL	18
+2	2	8
+2	3	10
+3	NULL	16
+3	2	16
+5	NULL	4
+5	2	4
+8	NULL	2
+8	1	2

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-2-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-3-59adefab34d80e8e185b2ad03877d381
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a
new file mode 100644
index 0000000..da4a754
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-4-9f501f293fe180bf6322e93d8dea025a
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-5-1163c486fd7e2c4346805fb035e2f268
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc
new file mode 100644
index 0000000..b20db4c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-6-9459dc3e8ab1f09d6d912b686e7f37fc
@@ -0,0 +1,15 @@
+NULL	NULL	23.0
+NULL	1	4.0
+NULL	2	14.0
+NULL	3	5.0
+1	NULL	3.0
+1	1	3.0
+2	NULL	9.0
+2	2	4.0
+2	3	5.0
+3	NULL	8.0
+3	2	8.0
+5	NULL	2.0
+5	2	2.0
+8	NULL	1.0
+8	1	1.0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-7-39db88427f92cb770b6daa38610c04e7
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786 b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-8-e464ec3d5461bda47eac3d1ef8617786
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff b/sql/hive/src/test/resources/golden/groupby_grouping_sets2-9-74126b100714164d13cbb3bff436c2ff
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-0-4fe85ca1e68a895ef403afdcfbbf61bc
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-38373b67d392924967a4695689d2164e
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-1-e8964b2aaeb388064c9fdac5ec687824
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-c1942a377b1a440d4ed3dd05fed445d
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-2-d95cf9dfae402d369f338b8516845e02
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-3-b89ea2173180c8ae423d856f943e061f
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-4-77c57e1b4ca37c2dc715b65668cd0c59
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-5-6623f95d90d929a6d2c8171a0698d4fd
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef
new file mode 100644
index 0000000..b2d0894
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-6-aec59088408cc57248851d3ce04e2eef
@@ -0,0 +1,16 @@
+NULL	NULL	3.8333333333333335	12
+NULL	1	2.0	5
+NULL	2	5.2	5
+NULL	3	5.0	2
+1	NULL	2.6666666666666665	3
+1	1	3.0	2
+1	2	2.0	1
+2	NULL	5.2	5
+2	2	5.333333333333333	3
+2	3	5.0	2
+3	NULL	8.0	1
+3	2	8.0	1
+5	NULL	2.0	1
+5	1	2.0	1
+8	NULL	1.0	2
+8	1	1.0	2

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-7-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966 b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-8-f32c4a191759237733a10cd721b49966
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef
new file mode 100644
index 0000000..b2d0894
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets3-9-aec59088408cc57248851d3ce04e2eef
@@ -0,0 +1,16 @@
+NULL	NULL	3.8333333333333335	12
+NULL	1	2.0	5
+NULL	2	5.2	5
+NULL	3	5.0	2
+1	NULL	2.6666666666666665	3
+1	1	3.0	2
+1	2	2.0	1
+2	NULL	5.2	5
+2	2	5.333333333333333	3
+2	3	5.0	2
+3	NULL	8.0	1
+3	2	8.0	1
+5	NULL	2.0	1
+5	1	2.0	1
+8	NULL	1.0	2
+8	1	1.0	2

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-0-d8ae5a3e613dd2dda392995b90d47565
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-1-6ad4e855adb49babfa3ae6abac190be3
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-2-1789808269c8bd0f6259227f07da1a6a
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-3-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-4-b51d813d0c5a410d8cf9765d85005a01
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438
new file mode 100644
index 0000000..33060f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-5-9be98faf8588a3c8e7436f14c638e438
@@ -0,0 +1,13 @@
+1	NULL	1	1	NULL	1
+1	NULL	1	1	1	1
+1	1	1	1	NULL	1
+1	1	1	1	1	1
+2	NULL	2	2	NULL	2
+2	NULL	2	2	2	1
+2	NULL	2	2	3	1
+2	2	1	2	NULL	2
+2	2	1	2	2	1
+2	2	1	2	3	1
+2	3	1	2	NULL	2
+2	3	1	2	2	1
+2	3	1	2	3	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-6-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-7-98f7522ce136cdffb0c14163e613c250
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438 b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438
new file mode 100644
index 0000000..33060f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets4-8-9be98faf8588a3c8e7436f14c638e438
@@ -0,0 +1,13 @@
+1	NULL	1	1	NULL	1
+1	NULL	1	1	1	1
+1	1	1	1	NULL	1
+1	1	1	1	1	1
+2	NULL	2	2	NULL	2
+2	NULL	2	2	2	1
+2	NULL	2	2	3	1
+2	2	1	2	NULL	2
+2	2	1	2	2	1
+2	2	1	2	3	1
+2	3	1	2	NULL	2
+2	3	1	2	2	1
+2	3	1	2	3	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-0-d8ae5a3e613dd2dda392995b90d47565
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-1-6ad4e855adb49babfa3ae6abac190be3
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-2-1789808269c8bd0f6259227f07da1a6a
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-3673d61d0944adeba77438d882839de4
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-3-d1c300ea08361fb8237689c6cf8cc1b5
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-4-6a89a34347f1c0eb9e0763ecedddb6f9
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f
new file mode 100644
index 0000000..da4a754
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-5-af3569757b9f52fb9b1ead33130e1b4f
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6
new file mode 100644
index 0000000..573541a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-6-bb96e87a0d6b0d1a6167e424b086acf6
@@ -0,0 +1 @@
+0

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8 b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-7-6544a382d851f916616c4386fdcf0ed8
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/spark/blob/f728e0fe/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f
new file mode 100644
index 0000000..da4a754
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_grouping_sets5-8-af3569757b9f52fb9b1ead33130e1b4f
@@ -0,0 +1,15 @@
+NULL	NULL	6
+NULL	1	2
+NULL	2	3
+NULL	3	1
+1	NULL	1
+1	1	1
+2	NULL	2
+2	2	1
+2	3	1
+3	NULL	1
+3	2	1
+5	NULL	1
+5	2	1
+8	NULL	1
+8	1	1


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org