You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2017/02/27 08:55:58 UTC

hive git commit: HIVE-15994: Grouping function error when grouping sets are not specified (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

Repository: hive
Updated Branches:
  refs/heads/master 3813f39b7 -> 13a1978a6


HIVE-15994: Grouping function error when grouping sets are not specified (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


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

Branch: refs/heads/master
Commit: 13a1978a6252f432d85dbffa5ab5dd6c04a9ad05
Parents: 3813f39
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Feb 21 14:05:01 2017 +0000
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Mon Feb 27 08:55:39 2017 +0000

----------------------------------------------------------------------
 .../hadoop/hive/ql/parse/CalcitePlanner.java    |  19 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  47 ++--
 .../groupby_grouping_sets_grouping.q            |  29 +++
 .../clientpositive/groupby_grouping_window.q    |  13 ++
 .../groupby_grouping_sets_grouping.q.out        | 231 +++++++++++++++++++
 .../groupby_grouping_window.q.out               |  48 ++++
 6 files changed, 360 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/13a1978a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 3a10cbf..db8d46e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -3356,10 +3356,11 @@ public class CalcitePlanner extends SemanticAnalyzer {
       final boolean cubeRollupGrpSetPresent = (!qbp.getDestRollups().isEmpty()
               || !qbp.getDestGroupingSets().isEmpty() || !qbp.getDestCubes().isEmpty());
       for (WindowExpressionSpec wExprSpec : windowExpressions) {
-        if (cubeRollupGrpSetPresent) {
+        if (!qbp.getDestToGroupBy().isEmpty()) {
           // Special handling of grouping function
           wExprSpec.setExpression(rewriteGroupingFunctionAST(
-                  getGroupByForClause(qbp, selClauseName), wExprSpec.getExpression()));
+                  getGroupByForClause(qbp, selClauseName), wExprSpec.getExpression(),
+                  !cubeRollupGrpSetPresent));
         }
         if (out_rwsch.getExpression(wExprSpec.getExpression()) == null) {
           Pair<RexNode, TypeInfo> wtp = genWindowingProj(qb, wExprSpec, srcRel);
@@ -3647,9 +3648,10 @@ public class CalcitePlanner extends SemanticAnalyzer {
           TypeCheckCtx tcCtx = new TypeCheckCtx(inputRR);
           // We allow stateful functions in the SELECT list (but nowhere else)
           tcCtx.setAllowStatefulFunctions(true);
-          if (cubeRollupGrpSetPresent) {
+          if (!qbp.getDestToGroupBy().isEmpty()) {
             // Special handling of grouping function
-            expr = rewriteGroupingFunctionAST(getGroupByForClause(qbp, selClauseName), expr);
+            expr = rewriteGroupingFunctionAST(getGroupByForClause(qbp, selClauseName), expr,
+                !cubeRollupGrpSetPresent);
           }
           ExprNodeDesc exp = genExprNodeDesc(expr, inputRR, tcCtx);
           String recommended = recommendName(exp, colAlias);
@@ -4030,11 +4032,12 @@ public class CalcitePlanner extends SemanticAnalyzer {
         }
         ASTNode targetNode = (ASTNode) havingClause.getChild(0);
         validateNoHavingReferenceToAlias(qb, targetNode);
-        final boolean cubeRollupGrpSetPresent = (!qbp.getDestRollups().isEmpty()
-                || !qbp.getDestGroupingSets().isEmpty() || !qbp.getDestCubes().isEmpty());
-        if (cubeRollupGrpSetPresent) {
+        if (!qbp.getDestToGroupBy().isEmpty()) {
+          final boolean cubeRollupGrpSetPresent = (!qbp.getDestRollups().isEmpty()
+                  || !qbp.getDestGroupingSets().isEmpty() || !qbp.getDestCubes().isEmpty());
           // Special handling of grouping function
-          targetNode = rewriteGroupingFunctionAST(getGroupByForClause(qbp, destClauseName), targetNode);
+          targetNode = rewriteGroupingFunctionAST(getGroupByForClause(qbp, destClauseName), targetNode,
+              !cubeRollupGrpSetPresent);
         }
         gbFilter = genFilterRelNode(qb, targetNode, srcRel, aliasToRel, null, null, true);
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/13a1978a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 83603af..0872e53 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -3039,16 +3039,15 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     }
     ASTNode condn = (ASTNode) havingExpr.getChild(0);
 
-    if (!isCBOExecuted()) {
+    if (!isCBOExecuted() && !qb.getParseInfo().getDestToGroupBy().isEmpty()) {
       // If CBO did not optimize the query, we might need to replace grouping function
       final String destClauseName = qb.getParseInfo().getClauseNames().iterator().next();
       final boolean cubeRollupGrpSetPresent = (!qb.getParseInfo().getDestRollups().isEmpty()
               || !qb.getParseInfo().getDestGroupingSets().isEmpty()
               || !qb.getParseInfo().getDestCubes().isEmpty());
-      if (cubeRollupGrpSetPresent) {
-        // Special handling of grouping function
-        condn = rewriteGroupingFunctionAST(getGroupByForClause(qb.getParseInfo(), destClauseName), condn);
-      }
+      // Special handling of grouping function
+      condn = rewriteGroupingFunctionAST(getGroupByForClause(qb.getParseInfo(), destClauseName), condn,
+          !cubeRollupGrpSetPresent);
     }
 
     /*
@@ -3061,7 +3060,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return output;
   }
 
-  protected static ASTNode rewriteGroupingFunctionAST(final List<ASTNode> grpByAstExprs, ASTNode targetNode) throws SemanticException {
+  protected static ASTNode rewriteGroupingFunctionAST(final List<ASTNode> grpByAstExprs, ASTNode targetNode,
+          final boolean noneSet) throws SemanticException {
     final MutableBoolean visited = new MutableBoolean(false);
     final MutableBoolean found = new MutableBoolean(false);
 
@@ -3083,10 +3083,19 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
             for (int i = 0; i < grpByAstExprs.size(); i++) {
               ASTNode grpByExpr = grpByAstExprs.get(i);
               if (grpByExpr.toStringTree().equals(c.toStringTree())) {
-                ASTNode child1 = (ASTNode) ParseDriver.adaptor.create(
-                        HiveParser.TOK_TABLE_OR_COL, "TOK_TABLE_OR_COL");
-                ParseDriver.adaptor.addChild(child1, ParseDriver.adaptor.create(
-                        HiveParser.Identifier, VirtualColumn.GROUPINGID.getName()));
+                ASTNode child1;
+                if (noneSet) {
+                  // Query does not contain CUBE, ROLLUP, or GROUPING SETS, and thus,
+                  // grouping should return 0
+                  child1 = (ASTNode) ParseDriver.adaptor.create(HiveParser.IntegralLiteral,
+                        String.valueOf(0));
+                } else {
+                  // We refer to grouping_id column
+                  child1 = (ASTNode) ParseDriver.adaptor.create(
+                          HiveParser.TOK_TABLE_OR_COL, "TOK_TABLE_OR_COL");
+                  ParseDriver.adaptor.addChild(child1, ParseDriver.adaptor.create(
+                          HiveParser.Identifier, VirtualColumn.GROUPINGID.getName()));
+                }
                 ASTNode child2 = (ASTNode) ParseDriver.adaptor.create(HiveParser.IntegralLiteral,
                         String.valueOf(IntMath.mod(-i-1, grpByAstExprs.size())));
                 root.setChild(1, child1);
@@ -4294,10 +4303,11 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         // We allow stateful functions in the SELECT list (but nowhere else)
         tcCtx.setAllowStatefulFunctions(true);
         tcCtx.setAllowDistinctFunctions(false);
-        if (!isCBOExecuted() && cubeRollupGrpSetPresent) {
+        if (!isCBOExecuted() && !qb.getParseInfo().getDestToGroupBy().isEmpty()) {
           // If CBO did not optimize the query, we might need to replace grouping function
           // Special handling of grouping function
-          expr = rewriteGroupingFunctionAST(getGroupByForClause(qb.getParseInfo(), dest), expr);
+          expr = rewriteGroupingFunctionAST(getGroupByForClause(qb.getParseInfo(), dest), expr,
+              !cubeRollupGrpSetPresent);
         }
         ExprNodeDesc exp = genExprNodeDesc(expr, inputRR, tcCtx);
         String recommended = recommendName(exp, colAlias);
@@ -13136,18 +13146,17 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   Operator genWindowingPlan(QB qb, WindowingSpec wSpec, Operator input) throws SemanticException {
     wSpec.validateAndMakeEffective();
 
-    if (!isCBOExecuted()) {
+    if (!isCBOExecuted() && !qb.getParseInfo().getDestToGroupBy().isEmpty()) {
       // If CBO did not optimize the query, we might need to replace grouping function
       final String selClauseName = qb.getParseInfo().getClauseNames().iterator().next();
       final boolean cubeRollupGrpSetPresent = (!qb.getParseInfo().getDestRollups().isEmpty()
               || !qb.getParseInfo().getDestGroupingSets().isEmpty()
               || !qb.getParseInfo().getDestCubes().isEmpty());
-      if (cubeRollupGrpSetPresent) {
-        for (WindowExpressionSpec wExprSpec : wSpec.getWindowExpressions()) {
-          // Special handling of grouping function
-          wExprSpec.setExpression(rewriteGroupingFunctionAST(
-                  getGroupByForClause(qb.getParseInfo(), selClauseName), wExprSpec.getExpression()));
-        }
+      for (WindowExpressionSpec wExprSpec : wSpec.getWindowExpressions()) {
+        // Special handling of grouping function
+        wExprSpec.setExpression(rewriteGroupingFunctionAST(
+                getGroupByForClause(qb.getParseInfo(), selClauseName), wExprSpec.getExpression(),
+                !cubeRollupGrpSetPresent));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/13a1978a/ql/src/test/queries/clientpositive/groupby_grouping_sets_grouping.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/groupby_grouping_sets_grouping.q b/ql/src/test/queries/clientpositive/groupby_grouping_sets_grouping.q
index 1b753e1..7856097 100644
--- a/ql/src/test/queries/clientpositive/groupby_grouping_sets_grouping.q
+++ b/ql/src/test/queries/clientpositive/groupby_grouping_sets_grouping.q
@@ -87,3 +87,32 @@ from T1
 group by cube(key, value)
 having grouping(key) = 1 OR grouping(value) = 1
 order by x desc, case when x = 1 then key end;
+
+explain
+select key, value, grouping(key), grouping(value)
+from T1
+group by key, value;
+
+select key, value, grouping(key), grouping(value)
+from T1
+group by key, value;
+
+explain
+select key, value, grouping(value)
+from T1
+group by key, value;
+
+select key, value, grouping(value)
+from T1
+group by key, value;
+
+explain
+select key, value
+from T1
+group by key, value
+having grouping(key) = 0;
+
+select key, value
+from T1
+group by key, value
+having grouping(key) = 0;

http://git-wip-us.apache.org/repos/asf/hive/blob/13a1978a/ql/src/test/queries/clientpositive/groupby_grouping_window.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/groupby_grouping_window.q b/ql/src/test/queries/clientpositive/groupby_grouping_window.q
index b456074..7ba73f9 100644
--- a/ql/src/test/queries/clientpositive/groupby_grouping_window.q
+++ b/ql/src/test/queries/clientpositive/groupby_grouping_window.q
@@ -13,3 +13,16 @@ FROM t
 GROUP BY category
 GROUPING SETS ((), (category))
 HAVING max(comments) > 0;
+
+SELECT grouping(category), lead(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live
+GROUPING SETS ((), (category));
+
+SELECT grouping(category), lead(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live;
+
+SELECT grouping(category), lag(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live;

http://git-wip-us.apache.org/repos/asf/hive/blob/13a1978a/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out b/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
index 62f40cd..6917dba 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_sets_grouping.q.out
@@ -765,3 +765,234 @@ NULL	2	1
 2	NULL	1
 3	NULL	1
 4	NULL	1
+PREHOOK: query: explain
+select key, value, grouping(key), grouping(value)
+from T1
+group by key, value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select key, value, grouping(key), grouping(value)
+from T1
+group by key, value
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: int), value (type: int)
+              outputColumnNames: key, value
+              Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                keys: key (type: int), value (type: int)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                  Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: int), KEY._col1 (type: int)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: int), _col1 (type: int), 0 (type: tinyint), 0 (type: tinyint)
+            outputColumnNames: _col0, _col1, _col2, _col3
+            Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select key, value, grouping(key), grouping(value)
+from T1
+group by key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: select key, value, grouping(key), grouping(value)
+from T1
+group by key, value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+#### A masked pattern was here ####
+1	NULL	0	0
+1	1	0	0
+2	2	0	0
+3	NULL	0	0
+3	3	0	0
+4	5	0	0
+PREHOOK: query: explain
+select key, value, grouping(value)
+from T1
+group by key, value
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select key, value, grouping(value)
+from T1
+group by key, value
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: int), value (type: int)
+              outputColumnNames: key, value
+              Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                keys: key (type: int), value (type: int)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                  Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: int), KEY._col1 (type: int)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+          Select Operator
+            expressions: _col0 (type: int), _col1 (type: int), 0 (type: tinyint)
+            outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+            File Output Operator
+              compressed: false
+              Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+              table:
+                  input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select key, value, grouping(value)
+from T1
+group by key, value
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: select key, value, grouping(value)
+from T1
+group by key, value
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+#### A masked pattern was here ####
+1	NULL	0
+1	1	0
+2	2	0
+3	NULL	0
+3	3	0
+4	5	0
+PREHOOK: query: explain
+select key, value
+from T1
+group by key, value
+having grouping(key) = 0
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+select key, value
+from T1
+group by key, value
+having grouping(key) = 0
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Map Operator Tree:
+          TableScan
+            alias: t1
+            Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+            Select Operator
+              expressions: key (type: int), value (type: int)
+              outputColumnNames: key, value
+              Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+              Group By Operator
+                keys: key (type: int), value (type: int)
+                mode: hash
+                outputColumnNames: _col0, _col1
+                Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col0 (type: int), _col1 (type: int)
+                  sort order: ++
+                  Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
+                  Statistics: Num rows: 3 Data size: 30 Basic stats: COMPLETE Column stats: NONE
+      Reduce Operator Tree:
+        Group By Operator
+          keys: KEY._col0 (type: int), KEY._col1 (type: int)
+          mode: mergepartial
+          outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+          File Output Operator
+            compressed: false
+            Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE
+            table:
+                input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+      Processor Tree:
+        ListSink
+
+PREHOOK: query: select key, value
+from T1
+group by key, value
+having grouping(key) = 0
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: select key, value
+from T1
+group by key, value
+having grouping(key) = 0
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+#### A masked pattern was here ####
+1	NULL
+1	1
+2	2
+3	NULL
+3	3
+4	5

http://git-wip-us.apache.org/repos/asf/hive/blob/13a1978a/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/groupby_grouping_window.q.out b/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
index 251f4f7..202fad0 100644
--- a/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
+++ b/ql/src/test/results/clientpositive/groupby_grouping_window.q.out
@@ -151,3 +151,51 @@ NULL	0	2	1
 86	0	2	1
 238	0	2	1
 311	0	2	1
+PREHOOK: query: SELECT grouping(category), lead(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live
+GROUPING SETS ((), (category))
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT grouping(category), lead(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live
+GROUPING SETS ((), (category))
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+0	NULL
+0	NULL
+0	NULL
+0	NULL
+PREHOOK: query: SELECT grouping(category), lead(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT grouping(category), lead(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+0	0
+0	0
+0	NULL
+PREHOOK: query: SELECT grouping(category), lag(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT grouping(category), lag(live) over(partition by grouping(category))
+FROM t
+GROUP BY category, live
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t
+#### A masked pattern was here ####
+0	NULL
+0	0
+0	0