You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jp...@apache.org on 2015/01/08 23:14:02 UTC
svn commit: r1650398 - in /hive/trunk/ql/src:
java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/
java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/
java/org/apache/hadoop/hive/ql/parse/ test/queries/clientpositive/
test/results/cl...
Author: jpullokk
Date: Thu Jan 8 22:14:01 2015
New Revision: 1650398
URL: http://svn.apache.org/r1650398
Log:
HIVE-8988 Support advanced aggregation in Hive to Calcite path (Jesus Camacho Rodriguez via Laljo John Pullokkaran)
Added:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveGroupingID.java
hive/trunk/ql/src/test/queries/clientpositive/groupby_grouping_sets6.q
hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out
Modified:
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
hive/trunk/ql/src/test/queries/clientpositive/groupby_cube1.q
hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
hive/trunk/ql/src/test/results/clientpositive/groupby_cube1.q.out
hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out
hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out
hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out
hive/trunk/ql/src/test/results/clientpositive/groupby_rollup1.q.out
hive/trunk/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveGroupingID.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveGroupingID.java?rev=1650398&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveGroupingID.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveGroupingID.java Thu Jan 8 22:14:01 2015
@@ -0,0 +1,25 @@
+package org.apache.hadoop.hive.ql.optimizer.calcite.reloperators;
+
+import org.apache.calcite.sql.SqlInternalOperator;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+public class HiveGroupingID extends SqlInternalOperator {
+
+ public static final SqlInternalOperator GROUPING__ID =
+ new HiveGroupingID();
+
+ private HiveGroupingID() {
+ super("$GROUPING__ID",
+ SqlKind.OTHER,
+ 0,
+ false,
+ ReturnTypes.BIGINT,
+ InferTypes.BOOLEAN,
+ OperandTypes.ONE_OR_MORE);
+ }
+
+}
+
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java Thu Jan 8 22:14:01 2015
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.RelFieldCo
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.RelVisitor;
import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Aggregate.Group;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Filter;
import org.apache.calcite.rel.core.Join;
@@ -49,12 +50,14 @@ import org.apache.calcite.rex.RexWindowB
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlOperator;
import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.util.BitSets;
+import org.apache.calcite.util.ImmutableBitSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSort;
import org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConverter.HiveToken;
import org.apache.hadoop.hive.ql.parse.ASTNode;
@@ -93,7 +96,7 @@ public class ASTConverter {
return c.convert();
}
- private ASTNode convert() {
+ private ASTNode convert() throws CalciteSemanticException {
/*
* 1. Walk RelNode Graph; note from, where, gBy.. nodes.
*/
@@ -118,15 +121,50 @@ public class ASTConverter {
* 4. GBy
*/
if (groupBy != null) {
- ASTBuilder b = ASTBuilder.construct(HiveParser.TOK_GROUPBY, "TOK_GROUPBY");
- for (int i : BitSets.toIter(groupBy.getGroupSet())) {
+ ASTBuilder b;
+ boolean groupingSetsExpression = false;
+ if (groupBy.indicator) {
+ Group aggregateType = Aggregate.Group.induce(groupBy.getGroupSet(),
+ groupBy.getGroupSets());
+ if (aggregateType == Group.ROLLUP) {
+ b = ASTBuilder.construct(HiveParser.TOK_ROLLUP_GROUPBY, "TOK_ROLLUP_GROUPBY");
+ }
+ else if (aggregateType == Group.CUBE) {
+ b = ASTBuilder.construct(HiveParser.TOK_CUBE_GROUPBY, "TOK_CUBE_GROUPBY");
+ }
+ else {
+ b = ASTBuilder.construct(HiveParser.TOK_GROUPING_SETS, "TOK_GROUPING_SETS");
+ groupingSetsExpression = true;
+ }
+ }
+ else {
+ b = ASTBuilder.construct(HiveParser.TOK_GROUPBY, "TOK_GROUPBY");
+ }
+
+ for (int i : groupBy.getGroupSet()) {
RexInputRef iRef = new RexInputRef(i, groupBy.getCluster().getTypeFactory()
.createSqlType(SqlTypeName.ANY));
b.add(iRef.accept(new RexVisitor(schema)));
}
- if (!groupBy.getGroupSet().isEmpty())
+ //Grouping sets expressions
+ if(groupingSetsExpression) {
+ for(ImmutableBitSet groupSet: groupBy.getGroupSets()) {
+ ASTBuilder expression = ASTBuilder.construct(
+ HiveParser.TOK_GROUPING_SETS_EXPRESSION, "TOK_GROUPING_SETS_EXPRESSION");
+ for (int i : groupSet) {
+ RexInputRef iRef = new RexInputRef(i, groupBy.getCluster().getTypeFactory()
+ .createSqlType(SqlTypeName.ANY));
+ expression.add(iRef.accept(new RexVisitor(schema)));
+ }
+ b.add(expression);
+ }
+ }
+
+ if (!groupBy.getGroupSet().isEmpty()) {
hiveAST.groupBy = b.node();
+ }
+
schema = new Schema(schema, groupBy);
}
@@ -151,9 +189,33 @@ public class ASTConverter {
int i = 0;
for (RexNode r : select.getChildExps()) {
- ASTNode selectExpr = ASTBuilder.selectExpr(r.accept(
- new RexVisitor(schema, r instanceof RexLiteral)),
- select.getRowType().getFieldNames().get(i++));
+ // If it is a GroupBy with grouping sets and grouping__id column
+ // is selected, we reformulate to project that column from
+ // the output of the GroupBy operator
+ boolean reformulate = false;
+ if (groupBy != null && groupBy.indicator) {
+ RexNode expr = select.getChildExps().get(i);
+ if (expr instanceof RexCall) {
+ if ( ((RexCall) expr).getOperator().
+ equals(HiveGroupingID.GROUPING__ID)) {
+ reformulate = true;
+ }
+ }
+ }
+ ASTNode expr;
+ if(reformulate) {
+ RexInputRef iRef = new RexInputRef(
+ groupBy.getGroupCount() * 2 + groupBy.getAggCallList().size(),
+ TypeConverter.convert(
+ VirtualColumn.GROUPINGID.getTypeInfo(),
+ groupBy.getCluster().getTypeFactory()));
+ expr = iRef.accept(new RexVisitor(schema));
+ }
+ else {
+ expr = r.accept(new RexVisitor(schema, r instanceof RexLiteral));
+ }
+ String alias = select.getRowType().getFieldNames().get(i++);
+ ASTNode selectExpr = ASTBuilder.selectExpr(expr, alias);
b.add(selectExpr);
}
}
@@ -232,7 +294,7 @@ public class ASTConverter {
return new Schema(select, tblAlias);
}
- private QueryBlockInfo convertSource(RelNode r) {
+ private QueryBlockInfo convertSource(RelNode r) throws CalciteSemanticException {
Schema s;
ASTNode ast;
@@ -554,10 +616,19 @@ public class ASTConverter {
}
Schema(Schema src, Aggregate gBy) {
- for (int i : BitSets.toIter(gBy.getGroupSet())) {
+ for (int i : gBy.getGroupSet()) {
ColumnInfo cI = src.get(i);
add(cI);
}
+ // If we are using grouping sets, we add the
+ // fields again, these correspond to the boolean
+ // grouping in Calcite. They are not used by Hive.
+ if(gBy.indicator) {
+ for (int i : gBy.getGroupSet()) {
+ ColumnInfo cI = src.get(i);
+ add(cI);
+ }
+ }
List<AggregateCall> aggs = gBy.getAggCallList();
for (AggregateCall agg : aggs) {
int argCount = agg.getArgList().size();
@@ -572,6 +643,9 @@ public class ASTConverter {
}
add(new ColumnInfo(null, b.node()));
}
+ if(gBy.indicator) {
+ add(new ColumnInfo(null,VirtualColumn.GROUPINGID.getName()));
+ }
}
/**
@@ -665,4 +739,5 @@ public class ASTConverter {
return flat;
}
+
}
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java Thu Jan 8 22:14:01 2015
@@ -22,6 +22,8 @@ import java.lang.reflect.InvocationTarge
import java.lang.reflect.UndeclaredThrowableException;
import java.math.BigDecimal;
import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -108,15 +110,16 @@ import org.apache.hadoop.hive.ql.exec.Op
import org.apache.hadoop.hive.ql.lib.Node;
import org.apache.hadoop.hive.ql.metadata.Table;
import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
-import org.apache.hadoop.hive.ql.optimizer.calcite.HiveDefaultRelMetadataProvider;
+import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveDefaultRelMetadataProvider;
import org.apache.hadoop.hive.ql.optimizer.calcite.HiveTypeSystemImpl;
-import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveVolcanoPlanner;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveRelNode;
@@ -161,9 +164,9 @@ import org.apache.hadoop.hive.serde2.typ
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableList.Builder;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
-import com.google.common.collect.ImmutableList.Builder;
public class CalcitePlanner extends SemanticAnalyzer {
private final AtomicInteger noColsMissingStats = new AtomicInteger(0);
@@ -1521,7 +1524,7 @@ public class CalcitePlanner extends Sema
}
private RelNode genGBRelNode(List<ExprNodeDesc> gbExprs, List<AggInfo> aggInfoLst,
- RelNode srcRel) throws SemanticException {
+ List<Integer> groupSets, RelNode srcRel) throws SemanticException {
ImmutableMap<String, Integer> posMap = this.relToHiveColNameCalcitePosMap.get(srcRel);
RexNodeConverter converter = new RexNodeConverter(this.cluster, srcRel.getRowType(), posMap,
0, false);
@@ -1553,10 +1556,25 @@ public class CalcitePlanner extends Sema
}
RelNode gbInputRel = HiveProject.create(srcRel, gbChildProjLst, null);
+ // Grouping sets: we need to transform them into ImmutableBitSet
+ // objects for Calcite
+ List<ImmutableBitSet> transformedGroupSets = null;
+ if(groupSets != null && !groupSets.isEmpty()) {
+ Set<ImmutableBitSet> setTransformedGroupSets =
+ new HashSet<ImmutableBitSet>(groupSets.size());
+ for(int val: groupSets) {
+ setTransformedGroupSets.add(convert(val));
+ }
+ // Calcite expects the grouping sets sorted and without duplicates
+ transformedGroupSets = new ArrayList<ImmutableBitSet>(setTransformedGroupSets);
+ Collections.sort(transformedGroupSets, ImmutableBitSet.COMPARATOR);
+ }
+
HiveRelNode aggregateRel = null;
try {
aggregateRel = new HiveAggregate(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
- gbInputRel, false, groupSet, null, aggregateCalls);
+ gbInputRel, (transformedGroupSets!=null ? true:false), groupSet,
+ transformedGroupSets, aggregateCalls);
} catch (InvalidRelException e) {
throw new SemanticException(e);
}
@@ -1564,6 +1582,19 @@ public class CalcitePlanner extends Sema
return aggregateRel;
}
+ private ImmutableBitSet convert(int value) {
+ BitSet bits = new BitSet();
+ int index = 0;
+ while (value != 0L) {
+ if (value % 2 != 0) {
+ bits.set(index);
+ }
+ ++index;
+ value = value >>> 1;
+ }
+ return ImmutableBitSet.FROM_BIT_SET.apply(bits);
+ }
+
private void addAlternateGByKeyMappings(ASTNode gByExpr, ColumnInfo colInfo,
RowResolver gByInputRR, RowResolver gByRR) {
if (gByExpr.getType() == HiveParser.DOT
@@ -1698,26 +1729,6 @@ public class CalcitePlanner extends Sema
RelNode gbRel = null;
QBParseInfo qbp = getQBParseInfo(qb);
- // 0. for GSets, Cube, Rollup, bail from Calcite path.
- if (!qbp.getDestRollups().isEmpty() || !qbp.getDestGroupingSets().isEmpty()
- || !qbp.getDestCubes().isEmpty()) {
- String gbyClause = null;
- HashMap<String, ASTNode> gbysMap = qbp.getDestToGroupBy();
- if (gbysMap.size() == 1) {
- ASTNode gbyAST = gbysMap.entrySet().iterator().next().getValue();
- gbyClause = ctx.getTokenRewriteStream().toString(gbyAST.getTokenStartIndex(),
- gbyAST.getTokenStopIndex());
- gbyClause = "in '" + gbyClause + "'.";
- } else {
- gbyClause = ".";
- }
- String msg = String.format("Encountered Grouping Set/Cube/Rollup%s"
- + " Currently we don't support Grouping Set/Cube/Rollup" + " clauses in CBO,"
- + " turn off cbo for these queries.", gbyClause);
- LOG.debug(msg);
- throw new CalciteSemanticException(msg);
- }
-
// 1. Gather GB Expressions (AST) (GB + Aggregations)
// NOTE: Multi Insert is not supported
String detsClauseName = qbp.getClauseNames().iterator().next();
@@ -1751,18 +1762,34 @@ public class CalcitePlanner extends Sema
}
}
- // 4. Construct aggregation function Info
+ // 4. GroupingSets, Cube, Rollup
+ int groupingColsSize = gbExprNDescLst.size();
+ List<Integer> groupingSets = null;
+ if (!qbp.getDestRollups().isEmpty()
+ || !qbp.getDestGroupingSets().isEmpty()
+ || !qbp.getDestCubes().isEmpty()) {
+ if (qbp.getDestRollups().contains(detsClauseName)) {
+ groupingSets = getGroupingSetsForRollup(grpByAstExprs.size());
+ } else if (qbp.getDestCubes().contains(detsClauseName)) {
+ groupingSets = getGroupingSetsForCube(grpByAstExprs.size());
+ } else if (qbp.getDestGroupingSets().contains(detsClauseName)) {
+ groupingSets = getGroupingSets(grpByAstExprs, qbp, detsClauseName);
+ }
+ groupingColsSize = groupingColsSize * 2;
+ }
+
+ // 5. Construct aggregation function Info
ArrayList<AggInfo> aggregations = new ArrayList<AggInfo>();
if (hasAggregationTrees) {
assert (aggregationTrees != null);
for (ASTNode value : aggregationTrees.values()) {
- // 4.1 Determine type of UDAF
+ // 5.1 Determine type of UDAF
// This is the GenericUDAF name
String aggName = SemanticAnalyzer.unescapeIdentifier(value.getChild(0).getText());
boolean isDistinct = value.getType() == HiveParser.TOK_FUNCTIONDI;
boolean isAllColumns = value.getType() == HiveParser.TOK_FUNCTIONSTAR;
- // 4.2 Convert UDAF Params to ExprNodeDesc
+ // 5.2 Convert UDAF Params to ExprNodeDesc
ArrayList<ExprNodeDesc> aggParameters = new ArrayList<ExprNodeDesc>();
for (int i = 1; i < value.getChildCount(); i++) {
ASTNode paraExpr = (ASTNode) value.getChild(i);
@@ -1779,18 +1806,62 @@ public class CalcitePlanner extends Sema
aggParameters);
AggInfo aInfo = new AggInfo(aggParameters, udaf.returnType, aggName, isDistinct);
aggregations.add(aInfo);
- String field = SemanticAnalyzer.getColumnInternalName(gbExprNDescLst.size()
- + aggregations.size() - 1);
+ String field = getColumnInternalName(groupingColsSize + aggregations.size() - 1);
outputColumnNames.add(field);
groupByOutputRowResolver.putExpression(value, new ColumnInfo(field, aInfo.m_returnType,
"", false));
}
}
- gbRel = genGBRelNode(gbExprNDescLst, aggregations, srcRel);
+ gbRel = genGBRelNode(gbExprNDescLst, aggregations, groupingSets, srcRel);
relToHiveColNameCalcitePosMap.put(gbRel,
buildHiveToCalciteColumnMap(groupByOutputRowResolver, gbRel));
this.relToHiveRR.put(gbRel, groupByOutputRowResolver);
+
+ // 6. If GroupingSets, Cube, Rollup were used, we account grouping__id.
+ // Further, we insert a project operator on top to remove the grouping
+ // boolean associated to each column in Calcite; this will avoid
+ // recalculating all column positions when we go back from Calcite to Hive
+ if(groupingSets != null && !groupingSets.isEmpty()) {
+ RowResolver selectOutputRowResolver = new RowResolver();
+ selectOutputRowResolver.setIsExprResolver(true);
+ RowResolver.add(selectOutputRowResolver, groupByOutputRowResolver);
+ outputColumnNames = new ArrayList<String>(outputColumnNames);
+
+ // 6.1 List of columns to keep from groupBy operator
+ List<RelDataTypeField> gbOutput = gbRel.getRowType().getFieldList();
+ List<RexNode> calciteColLst = new ArrayList<RexNode>();
+ for(RelDataTypeField gbOut: gbOutput) {
+ if(gbOut.getIndex() < gbExprNDescLst.size() ||
+ gbOut.getIndex() >= gbExprNDescLst.size() * 2) {
+ calciteColLst.add(new RexInputRef(gbOut.getIndex(), gbOut.getType()));
+ }
+ }
+
+ // 6.2 Add column for grouping_id function
+ String field = getColumnInternalName(groupingColsSize + aggregations.size());
+ outputColumnNames.add(field);
+ selectOutputRowResolver.put(null, VirtualColumn.GROUPINGID.getName(),
+ new ColumnInfo(
+ field,
+ TypeInfoFactory.stringTypeInfo,
+ null,
+ true));
+
+ // 6.3 Compute column for grouping_id function in Calcite
+ List<RexNode> identifierCols = new ArrayList<RexNode>();
+ for(int i = gbExprNDescLst.size(); i < gbExprNDescLst.size() * 2; i++) {
+ identifierCols.add(new RexInputRef(
+ i, gbOutput.get(i).getType()));
+ }
+ final RexBuilder rexBuilder = cluster.getRexBuilder();
+ RexNode groupingID = rexBuilder.makeCall(HiveGroupingID.GROUPING__ID,
+ identifierCols);
+ calciteColLst.add(groupingID);
+
+ // Create select
+ gbRel = this.genSelectRelNode(calciteColLst, selectOutputRowResolver, gbRel);
+ }
}
return gbRel;
Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Thu Jan 8 22:14:01 2015
@@ -3117,7 +3117,7 @@ public class SemanticAnalyzer extends Ba
}
}
- private List<Integer> getGroupingSetsForRollup(int size) {
+ protected List<Integer> getGroupingSetsForRollup(int size) {
List<Integer> groupingSetKeys = new ArrayList<Integer>();
for (int i = 0; i <= size; i++) {
groupingSetKeys.add((1 << i) - 1);
@@ -3125,7 +3125,7 @@ public class SemanticAnalyzer extends Ba
return groupingSetKeys;
}
- private List<Integer> getGroupingSetsForCube(int size) {
+ protected List<Integer> getGroupingSetsForCube(int size) {
int count = 1 << size;
List<Integer> results = new ArrayList<Integer>(count);
for (int i = 0; i < count; ++i) {
@@ -3152,7 +3152,7 @@ public class SemanticAnalyzer extends Ba
return new ObjectPair<List<ASTNode>, List<Integer>>(groupByExprs, groupingSets);
}
- private List<Integer> getGroupingSets(List<ASTNode> groupByExpr, QBParseInfo parseInfo,
+ protected List<Integer> getGroupingSets(List<ASTNode> groupByExpr, QBParseInfo parseInfo,
String dest) throws SemanticException {
Map<String, Integer> exprPos = new HashMap<String, Integer>();
for (int i = 0; i < groupByExpr.size(); ++i) {
Modified: hive/trunk/ql/src/test/queries/clientpositive/groupby_cube1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/groupby_cube1.q?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/groupby_cube1.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/groupby_cube1.q Thu Jan 8 22:14:01 2015
@@ -13,6 +13,11 @@ SELECT key, val, count(1) FROM T1 GROUP
SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube;
EXPLAIN
+SELECT key, val, GROUPING__ID, count(1) FROM T1 GROUP BY key, val with cube;
+
+SELECT key, val, GROUPING__ID, count(1) FROM T1 GROUP BY key, val with cube;
+
+EXPLAIN
SELECT key, count(distinct val) FROM T1 GROUP BY key with cube;
SELECT key, count(distinct val) FROM T1 GROUP BY key with cube;
Added: hive/trunk/ql/src/test/queries/clientpositive/groupby_grouping_sets6.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/groupby_grouping_sets6.q?rev=1650398&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/groupby_grouping_sets6.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/groupby_grouping_sets6.q Thu Jan 8 22:14:01 2015
@@ -0,0 +1,28 @@
+CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1;
+
+set hive.optimize.ppd = false;
+set hive.cbo.enable = false;
+
+-- This filter is not pushed down
+EXPLAIN
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5;
+
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5;
+
+set hive.cbo.enable = true;
+
+-- This filter is pushed down through aggregate with grouping sets by Calcite
+EXPLAIN
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5;
+
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5;
Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby.q.out Thu Jan 8 22:14:01 2015
@@ -373,10 +373,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 32 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
@@ -430,10 +430,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 24 Data size: 4200 Basic stats: COMPLETE Column stats: COMPLETE
@@ -487,10 +487,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 8 Data size: 1400 Basic stats: COMPLETE Column stats: COMPLETE
@@ -544,10 +544,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 16 Data size: 2800 Basic stats: COMPLETE Column stats: COMPLETE
@@ -601,10 +601,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 24 Data size: 4200 Basic stats: COMPLETE Column stats: COMPLETE
@@ -658,10 +658,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 32 Data size: 5600 Basic stats: COMPLETE Column stats: COMPLETE
@@ -776,10 +776,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 16 Data size: 2800 Basic stats: COMPLETE Column stats: COMPLETE
@@ -892,10 +892,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 32 Data size: 3184 Basic stats: COMPLETE Column stats: NONE
@@ -949,10 +949,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 24 Data size: 2388 Basic stats: COMPLETE Column stats: NONE
@@ -1006,10 +1006,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
@@ -1063,10 +1063,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 16 Data size: 1592 Basic stats: COMPLETE Column stats: NONE
@@ -1120,10 +1120,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 24 Data size: 2388 Basic stats: COMPLETE Column stats: NONE
@@ -1177,10 +1177,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 32 Data size: 3184 Basic stats: COMPLETE Column stats: NONE
@@ -1291,10 +1291,10 @@ STAGE PLANS:
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), locid (type: int)
- outputColumnNames: state, locid
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 8 Data size: 796 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), locid (type: int), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: int), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 32 Data size: 3184 Basic stats: COMPLETE Column stats: NONE
Modified: hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/annotate_stats_groupby2.q.out Thu Jan 8 22:14:01 2015
@@ -148,10 +148,10 @@ STAGE PLANS:
Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: state (type: string), country (type: string)
- outputColumnNames: state, country
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- keys: state (type: string), country (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 80 Data size: 800 Basic stats: COMPLETE Column stats: NONE
@@ -325,10 +325,10 @@ STAGE PLANS:
Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), country (type: string)
- outputColumnNames: state, country
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), country (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 32 Data size: 8256 Basic stats: COMPLETE Column stats: COMPLETE
@@ -439,10 +439,10 @@ STAGE PLANS:
Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
Select Operator
expressions: state (type: string), country (type: string)
- outputColumnNames: state, country
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 20 Data size: 200 Basic stats: COMPLETE Column stats: COMPLETE
Group By Operator
- keys: state (type: string), country (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 80 Data size: 20640 Basic stats: COMPLETE Column stats: COMPLETE
Modified: hive/trunk/ql/src/test/results/clientpositive/groupby_cube1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/groupby_cube1.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/groupby_cube1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/groupby_cube1.q.out Thu Jan 8 22:14:01 2015
@@ -37,11 +37,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: string), val (type: string)
- outputColumnNames: key, val
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: count(1)
- keys: key (type: string), val (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -103,6 +103,90 @@ NULL 18 1
NULL 28 1
NULL NULL 6
PREHOOK: query: EXPLAIN
+SELECT key, val, GROUPING__ID, count(1) FROM T1 GROUP BY key, val with cube
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT key, val, GROUPING__ID, count(1) FROM T1 GROUP BY key, val with cube
+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: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: key (type: string), val (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: count(1)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+ sort order: +++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ value expressions: _col3 (type: bigint)
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: count(VALUE._col0)
+ keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: SELECT key, val, GROUPING__ID, count(1) FROM T1 GROUP BY key, val with cube
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT key, val, GROUPING__ID, count(1) FROM T1 GROUP BY key, val with cube
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+#### A masked pattern was here ####
+1 11 3 1
+1 NULL 1 1
+2 12 3 1
+2 NULL 1 1
+3 13 3 1
+3 NULL 1 1
+7 17 3 1
+7 NULL 1 1
+8 18 3 1
+8 28 3 1
+8 NULL 1 2
+NULL 11 2 1
+NULL 12 2 1
+NULL 13 2 1
+NULL 17 2 1
+NULL 18 2 1
+NULL 28 2 1
+NULL NULL 0 6
+PREHOOK: query: EXPLAIN
SELECT key, count(distinct val) FROM T1 GROUP BY key with cube
PREHOOK: type: QUERY
POSTHOOK: query: EXPLAIN
@@ -193,11 +277,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: string), val (type: string)
- outputColumnNames: key, val
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: count(1)
- keys: key (type: string), val (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
Modified: hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets2.q.out Thu Jan 8 22:14:01 2015
@@ -36,11 +36,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: a (type: string), b (type: string)
- outputColumnNames: a, b
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: count()
- keys: a (type: string), b (type: string)
+ keys: _col0 (type: string), _col1 (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -142,11 +142,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: a (type: string), b (type: string), c (type: string)
- outputColumnNames: a, b, c
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Group By Operator
- aggregations: sum(c)
- keys: a (type: string), b (type: string)
+ aggregations: sum(_col2)
+ keys: _col0 (type: string), _col1 (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -269,12 +269,12 @@ STAGE PLANS:
alias: t2
Statistics: Num rows: 6 Data size: 42 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: a (type: string), b (type: string), c (type: int), d (type: int)
- outputColumnNames: a, b, c, d
+ expressions: a (type: string), b (type: string), (c + d) (type: int)
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 6 Data size: 42 Basic stats: COMPLETE Column stats: NONE
Group By Operator
- aggregations: sum((c + d))
- keys: a (type: string), b (type: string)
+ aggregations: sum(_col2)
+ keys: _col0 (type: string), _col1 (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 6 Data size: 42 Basic stats: COMPLETE Column stats: NONE
Modified: hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets3.q.out Thu Jan 8 22:14:01 2015
@@ -55,11 +55,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 72 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: a (type: string), b (type: string), c (type: string)
- outputColumnNames: a, b, c
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 0 Data size: 72 Basic stats: PARTIAL Column stats: NONE
Group By Operator
- aggregations: avg(c), count()
- keys: a (type: string), b (type: string), '0' (type: string)
+ aggregations: avg(_col2), count()
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3, _col4
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -142,11 +142,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 72 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: a (type: string), b (type: string), c (type: string)
- outputColumnNames: a, b, c
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 0 Data size: 72 Basic stats: PARTIAL Column stats: NONE
Group By Operator
- aggregations: avg(c), count()
- keys: a (type: string), b (type: string)
+ aggregations: avg(_col2), count()
+ keys: _col0 (type: string), _col1 (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
Modified: hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets5.q.out Thu Jan 8 22:14:01 2015
@@ -42,43 +42,36 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: a (type: string), b (type: string)
- outputColumnNames: a, b
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Group By Operator
- aggregations: count(1)
- keys: a (type: string), b (type: string)
+ keys: _col0 (type: string), _col1 (type: string)
mode: hash
- outputColumnNames: _col0, _col1, _col2
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: string), _col1 (type: string)
sort order: ++
Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- value expressions: _col2 (type: bigint)
Reduce Operator Tree:
Group By Operator
- aggregations: count(VALUE._col0)
keys: KEY._col0 (type: string), KEY._col1 (type: string)
mode: mergepartial
- outputColumnNames: _col0, _col1, _col2
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- Select Operator
- expressions: _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1
+ Group By Operator
+ aggregations: count()
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- Group By Operator
- aggregations: count()
- keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
- mode: hash
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
Stage: Stage-2
Map Reduce
@@ -165,43 +158,36 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: a (type: string), b (type: string)
- outputColumnNames: a, b
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
Group By Operator
- aggregations: count(1)
- keys: a (type: string), b (type: string)
+ keys: _col0 (type: string), _col1 (type: string)
mode: hash
- outputColumnNames: _col0, _col1, _col2
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: string), _col1 (type: string)
sort order: ++
Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- value expressions: _col2 (type: bigint)
Reduce Operator Tree:
Group By Operator
- aggregations: count(VALUE._col0)
keys: KEY._col0 (type: string), KEY._col1 (type: string)
mode: mergepartial
- outputColumnNames: _col0, _col1, _col2
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- Select Operator
- expressions: _col0 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1
+ Group By Operator
+ aggregations: count()
+ keys: _col0 (type: string), _col1 (type: string)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- Group By Operator
- aggregations: count()
- keys: _col0 (type: string), _col1 (type: string)
- mode: hash
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
Stage: Stage-2
Map Reduce
Added: hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out?rev=1650398&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/groupby_grouping_sets6.q.out Thu Jan 8 22:14:01 2015
@@ -0,0 +1,176 @@
+PREHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@T1
+POSTHOOK: query: CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@T1
+PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@t1
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@t1
+PREHOOK: query: -- This filter is not pushed down
+EXPLAIN
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+PREHOOK: type: QUERY
+POSTHOOK: query: -- This filter is not pushed down
+EXPLAIN
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+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: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: a (type: string), b (type: string)
+ outputColumnNames: a, b
+ Statistics: Num rows: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ keys: a (type: string), b (type: string), '0' (type: string)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+ sort order: +++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Filter Operator
+ predicate: (_col0 = 5) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: '5' (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+#### A masked pattern was here ####
+5 NULL
+5 2
+PREHOOK: query: -- This filter is pushed down through aggregate with grouping sets by Calcite
+EXPLAIN
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+PREHOOK: type: QUERY
+POSTHOOK: query: -- This filter is pushed down through aggregate with grouping sets by Calcite
+EXPLAIN
+SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+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: 0 Data size: 36 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: (UDFToDouble(a) = 5.0) (type: boolean)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: '5' (type: string), b (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Group By Operator
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+ sort order: +++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Reduce Operator Tree:
+ Group By Operator
+ keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT a, b FROM
+(SELECT a, b from T1 group by a, b grouping sets ( (a,b),a )) res
+WHERE res.a=5
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+#### A masked pattern was here ####
+5 NULL
+5 2
Modified: hive/trunk/ql/src/test/results/clientpositive/groupby_rollup1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/groupby_rollup1.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/groupby_rollup1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/groupby_rollup1.q.out Thu Jan 8 22:14:01 2015
@@ -37,11 +37,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: string), val (type: string)
- outputColumnNames: key, val
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: count(1)
- keys: key (type: string), val (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
@@ -187,11 +187,11 @@ STAGE PLANS:
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Select Operator
expressions: key (type: string), val (type: string)
- outputColumnNames: key, val
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE
Group By Operator
aggregations: count(1)
- keys: key (type: string), val (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
Modified: hive/trunk/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out?rev=1650398&r1=1650397&r2=1650398&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/infer_bucket_sort_grouping_operators.q.out Thu Jan 8 22:14:01 2015
@@ -44,11 +44,11 @@ STAGE PLANS:
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
- outputColumnNames: key, value
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count(1)
- keys: key (type: string), value (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1500 Data size: 15936 Basic stats: COMPLETE Column stats: NONE
@@ -231,11 +231,11 @@ STAGE PLANS:
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
- outputColumnNames: key, value
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count(1)
- keys: key (type: string), value (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
@@ -418,11 +418,11 @@ STAGE PLANS:
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
- outputColumnNames: key, value
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Group By Operator
aggregations: count(1)
- keys: key (type: string), value (type: string), '0' (type: string)
+ keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
mode: hash
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE