diff --git hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out index 27446b4..dca0e72 100644 --- hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out +++ hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out @@ -440,7 +440,7 @@ STAGE PLANS: alias: hbase_pushdown Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: (key >= '90') (type: boolean) + predicate: (not (key < '90')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: key (type: string), value (type: string) diff --git hbase-handler/src/test/results/positive/hbase_pushdown.q.out hbase-handler/src/test/results/positive/hbase_pushdown.q.out index a42e36f..77aa8e4 100644 --- hbase-handler/src/test/results/positive/hbase_pushdown.q.out +++ hbase-handler/src/test/results/positive/hbase_pushdown.q.out @@ -312,7 +312,7 @@ STAGE PLANS: alias: hbase_pushdown Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: (key <> 90) (type: boolean) + predicate: (not (key = 90)) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: key (type: int), value (type: string) diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java index 37dbe32..aa738a5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/NonBlockingOpDeDupProc.java @@ -28,7 +28,6 @@ import java.util.Set; import java.util.Stack; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.FilterOperator; import org.apache.hadoop.hive.ql.exec.JoinOperator; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; @@ -109,7 +108,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, if (cSEL.getColumnExprMap() == null) { // If the child SelectOperator does not have the ColumnExprMap, // we do not need to update the ColumnExprMap in the parent SelectOperator. - pSEL.getConf().setColList(ExprNodeDescUtils.backtrack(cSELColList, cSEL, pSEL)); + pSEL.getConf().setColList(ExprNodeDescUtils.backtrack(cSELColList, cSEL, pSEL, + pctx.getContext().isCboSucceeded())); pSEL.getConf().setOutputColumnNames(cSELOutputColumnNames); } else { // If the child SelectOperator has the ColumnExprMap, @@ -121,7 +121,8 @@ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx, String outputColumnName = cSELOutputColumnNames.get(i); ExprNodeDesc cSELExprNodeDesc = cSELColList.get(i); ExprNodeDesc newPSELExprNodeDesc = - ExprNodeDescUtils.backtrack(cSELExprNodeDesc, cSEL, pSEL); + ExprNodeDescUtils.backtrack(cSELExprNodeDesc, cSEL, pSEL, + pctx.getContext().isCboSucceeded()); newPSELColList.add(newPSELExprNodeDesc); newPSELOutputColumnNames.add(outputColumnName); colExprMap.put(outputColumnName, newPSELExprNodeDesc); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java index 55c71dd..062eadf 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java @@ -129,13 +129,11 @@ public void initialize(HiveConf hiveConf) { /* Add list bucketing pruner. */ transformations.add(new ListBucketingPruner()); } - } - if ((HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTPPD) - && HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION)) || - (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION) - && pctx.getContext().isCboSucceeded())) { - // PartitionPruner may create more folding opportunities, run ConstantPropagate again. - transformations.add(new ConstantPropagate()); + if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTCONSTANTPROPAGATION) && + !pctx.getContext().isCboSucceeded()) { + // PartitionPruner may create more folding opportunities, run ConstantPropagate again. + transformations.add(new ConstantPropagate()); + } } if (HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTGROUPBY) || diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java index adfbb67..a20db01 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java @@ -55,6 +55,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.ListBucketingCtx; @@ -287,7 +288,7 @@ private boolean allStaticPartitions(Operator op, } if (op.getColumnExprMap() != null) { for(String dpCol : dpCols) { - ExprNodeDesc end = findConstantExprOrigin(dpCol, op); + ExprNodeDesc end = ExprNodeDescUtils.findConstantExprOrigin(dpCol, op); if (!(end instanceof ExprNodeConstantDesc)) { return false; } @@ -298,37 +299,6 @@ private boolean allStaticPartitions(Operator op, return true; } - // Find the constant origin of a certain column if it is originated from a constant - // Otherwise, it returns the expression that originated the column - private ExprNodeDesc findConstantExprOrigin(String dpCol, Operator op) { - ExprNodeDesc expr = op.getColumnExprMap().get(dpCol); - ExprNodeDesc foldedExpr; - // If it is a function, we try to fold it - if (expr instanceof ExprNodeGenericFuncDesc) { - foldedExpr = ConstantPropagateProcFactory.foldExpr((ExprNodeGenericFuncDesc)expr); - if (foldedExpr == null) { - foldedExpr = expr; - } - } else { - foldedExpr = expr; - } - // If it is a column reference, we will try to resolve it - if (foldedExpr instanceof ExprNodeColumnDesc) { - Operator originOp = null; - for(Operator parentOp : op.getParentOperators()) { - if (parentOp.getColumnExprMap() != null) { - originOp = parentOp; - break; - } - } - if (originOp != null) { - return findConstantExprOrigin(((ExprNodeColumnDesc)foldedExpr).getColumn(), originOp); - } - } - // Otherwise, we return the expression - return foldedExpr; - } - // Remove RS and SEL introduced by enforce bucketing/sorting config // Convert PARENT -> RS -> SEL -> FS to PARENT -> FS private boolean removeRSInsertedByEnforceBucketing(FileSinkOperator fsOp) { diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java index 0cfd529..7febfd5 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java @@ -22,7 +22,6 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -30,8 +29,6 @@ import java.util.Set; import java.util.Stack; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData; @@ -67,6 +64,7 @@ import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; import org.apache.hadoop.hive.ql.plan.FetchWork; import org.apache.hadoop.hive.ql.plan.GroupByDesc; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount; @@ -81,6 +79,8 @@ import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.common.collect.Lists; @@ -213,13 +213,17 @@ private Long getNullcountFor(StatType type, ColumnStatisticsData statData) { private boolean hasNullOrConstantGbyKey(GroupByOperator gbyOp) { GroupByDesc gbyDesc = gbyOp.getConf(); + int numCols = gbyDesc.getOutputColumnNames().size(); + int aggCols = gbyDesc.getAggregators().size(); // If the Group by operator has null key - if (gbyDesc.getOutputColumnNames().size() == - gbyDesc.getAggregators().size()) { + if (numCols == aggCols) { return true; } - for (ExprNodeDesc en :gbyDesc.getKeys()) { - if (!(en instanceof ExprNodeConstantDesc)) { + // If the Gby key is a constant + List dpCols = gbyOp.getSchema().getColumnNames().subList(0, numCols - aggCols); + for(String dpCol : dpCols) { + ExprNodeDesc end = ExprNodeDescUtils.findConstantExprOrigin(dpCol, gbyOp); + if (!(end instanceof ExprNodeConstantDesc)) { return false; } } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java index 2f309f3..17f1b06 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HiveRexUtil.java @@ -19,8 +19,11 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.calcite.linq4j.Ord; @@ -29,12 +32,16 @@ import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; import org.apache.calcite.rex.RexUtil; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Pair; import org.apache.calcite.util.Util; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -42,6 +49,8 @@ public class HiveRexUtil { + protected static final Logger LOG = LoggerFactory.getLogger(HiveRexUtil.class); + /** * Simplifies a boolean expression. * @@ -54,13 +63,20 @@ * */ public static RexNode simplify(RexBuilder rexBuilder, RexNode e) { + return simplify(rexBuilder, e, false); + } + + public static RexNode simplify(RexBuilder rexBuilder, RexNode e, + boolean unknownAsFalse) { switch (e.getKind()) { case AND: - return simplifyAnd(rexBuilder, (RexCall) e); + return simplifyAnd(rexBuilder, (RexCall) e, unknownAsFalse); case OR: return simplifyOr(rexBuilder, (RexCall) e); + case NOT: + return simplifyNot(rexBuilder, (RexCall) e); case CASE: - return simplifyCase(rexBuilder, (RexCall) e); + return simplifyCase(rexBuilder, (RexCall) e, unknownAsFalse); case IS_NULL: return ((RexCall) e).getOperands().get(0).getType().isNullable() ? e : rexBuilder.makeLiteral(false); @@ -72,9 +88,27 @@ public static RexNode simplify(RexBuilder rexBuilder, RexNode e) { } } - private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call) { + private static RexNode simplifyNot(RexBuilder rexBuilder, RexCall call) { + final RexNode a = call.getOperands().get(0); + switch (a.getKind()) { + case NOT: + // NOT NOT x ==> x + return simplify(rexBuilder, ((RexCall) a).getOperands().get(0)); + } + final SqlKind negateKind = a.getKind().negate(); + if (a.getKind() != negateKind) { + return simplify(rexBuilder, + rexBuilder.makeCall(op(negateKind), + ImmutableList.of(((RexCall) a).getOperands().get(0)))); + } + return call; + } + + private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call, + boolean unknownAsFalse) { final List operands = call.getOperands(); final List newOperands = new ArrayList<>(); + final Set values = new HashSet<>(); for (int i = 0; i < operands.size(); i++) { RexNode operand = operands.get(i); if (RexUtil.isCasePredicate(call, i)) { @@ -88,13 +122,18 @@ private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call) { ++i; continue; } + } else { + if (unknownAsFalse && RexUtil.isNull(operand)) { + values.add(rexBuilder.makeLiteral(false).toString()); + } else { + values.add(operand.toString()); + } } newOperands.add(operand); } assert newOperands.size() % 2 == 1; - switch (newOperands.size()) { - case 1: - return rexBuilder.makeCast(call.getType(), newOperands.get(0)); + if (newOperands.size() == 1 || values.size() == 1) { + return rexBuilder.makeCast(call.getType(), newOperands.get(newOperands.size() - 1)); } trueFalse: if (call.getType().getSqlTypeName() == SqlTypeName.BOOLEAN) { @@ -110,7 +149,8 @@ private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call) { casePairs(rexBuilder, newOperands); for (Ord> pair : Ord.zip(pairs)) { if (!pair.e.getValue().isAlwaysTrue() - && !pair.e.getValue().isAlwaysFalse()) { + && !pair.e.getValue().isAlwaysFalse() + && (!unknownAsFalse || !RexUtil.isNull(pair.e.getValue()))) { break trueFalse; } } @@ -145,33 +185,76 @@ private static RexNode simplifyCase(RexBuilder rexBuilder, RexCall call) { return builder.build(); } - public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e) { - final List terms = RelOptUtil.conjunctions(e); + public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e, + boolean unknownAsFalse) { + final List terms = new ArrayList<>(); final List notTerms = new ArrayList<>(); - final List negatedTerms = new ArrayList<>(); - final List nullOperands = new ArrayList<>(); - final List notNullOperands = new ArrayList<>(); - final Set comparedOperands = new HashSet<>(); + RelOptUtil.decomposeConjunction(e, terms, notTerms); + if (unknownAsFalse) { + return simplifyAnd2ForUnknownAsFalse(rexBuilder, terms, notTerms); + } + return simplifyAnd2(rexBuilder, terms, notTerms); + } + + public static RexNode simplifyAnd2(RexBuilder rexBuilder, + List terms, List notTerms) { + if (terms.contains(rexBuilder.makeLiteral(false))) { + return rexBuilder.makeLiteral(false); + } + if (terms.isEmpty() && notTerms.isEmpty()) { + return rexBuilder.makeLiteral(true); + } + if (terms.size() == 1 && notTerms.isEmpty()) { + // Make sure "x OR y OR x" (a single-term conjunction) gets simplified. + return simplify(rexBuilder, terms.get(0)); + } + // If one of the not-disjunctions is a disjunction that is wholly + // contained in the disjunctions list, the expression is not + // satisfiable. + // + // Example #1. x AND y AND z AND NOT (x AND y) - not satisfiable + // Example #2. x AND y AND NOT (x AND y) - not satisfiable + // Example #3. x AND y AND NOT (x AND y AND z) - may be satisfiable + for (RexNode notDisjunction : notTerms) { + final List terms2 = RelOptUtil.conjunctions(notDisjunction); + if (terms.containsAll(terms2)) { + return rexBuilder.makeLiteral(false); + } + } + // Add the NOT disjunctions back in. + for (RexNode notDisjunction : notTerms) { + terms.add( + simplify(rexBuilder, + rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction))); + } + return RexUtil.composeConjunction(rexBuilder, terms, false); + } + + /** As {@link #simplifyAnd2(RexBuilder, List, List)} but we assume that if the expression returns + * UNKNOWN it will be interpreted as FALSE. */ + public static RexNode simplifyAnd2ForUnknownAsFalse(RexBuilder rexBuilder, + List terms, List notTerms) { + if (terms.contains(rexBuilder.makeLiteral(false))) { + return rexBuilder.makeLiteral(false); + } + if (terms.isEmpty() && notTerms.isEmpty()) { + return rexBuilder.makeLiteral(true); + } + if (terms.size() == 1 && notTerms.isEmpty()) { + // Make sure "x OR y OR x" (a single-term conjunction) gets simplified. + return simplify(rexBuilder, terms.get(0), true); + } + // Try to simplify the expression + final Set negatedTerms = new HashSet<>(); + final Set nullOperands = new HashSet<>(); + final Set notNullOperands = new LinkedHashSet<>(); + final Set comparedOperands = new HashSet<>(); for (int i = 0; i < terms.size(); i++) { final RexNode term = terms.get(i); if (!HiveCalciteUtil.isDeterministic(term)) { continue; } switch (term.getKind()) { - case NOT: - notTerms.add( - ((RexCall) term).getOperands().get(0)); - terms.remove(i); - --i; - break; - case LITERAL: - if (!RexLiteral.booleanValue(term)) { - return term; // false - } else { - terms.remove(i); - --i; - } - break; case EQUALS: case NOT_EQUALS: case LESS_THAN: @@ -180,53 +263,48 @@ public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e) { case GREATER_THAN_OR_EQUAL: RexCall call = (RexCall) term; RexNode left = call.getOperands().get(0); - comparedOperands.add(left); + comparedOperands.add(left.toString()); // if it is a cast, we include the inner reference if (left.getKind() == SqlKind.CAST) { RexCall leftCast = (RexCall) left; - comparedOperands.add(leftCast.getOperands().get(0)); + comparedOperands.add(leftCast.getOperands().get(0).toString()); } RexNode right = call.getOperands().get(1); - comparedOperands.add(right); + comparedOperands.add(right.toString()); // if it is a cast, we include the inner reference if (right.getKind() == SqlKind.CAST) { RexCall rightCast = (RexCall) right; - comparedOperands.add(rightCast.getOperands().get(0)); + comparedOperands.add(rightCast.getOperands().get(0).toString()); } - // Assume we have the expression a > 5. - // Then we can derive the negated term: NOT(a <= 5). + // Assume the expression a > 5 is part of a Filter condition. + // Then we can derive the negated term: a <= 5. // But as the comparison is string based and thus operands order dependent, - // we should also add the inverted negated term: NOT(5 >= a). + // we should also add the inverted negated term: 5 >= a. // Observe that for creating the inverted term we invert the list of operands. - RexCall negatedTerm = negate(rexBuilder, call); + RexNode negatedTerm = negate(rexBuilder, call); if (negatedTerm != null) { - negatedTerms.add(negatedTerm); - RexCall invertNegatedTerm = invert(rexBuilder, negatedTerm); + negatedTerms.add(negatedTerm.toString()); + RexNode invertNegatedTerm = invert(rexBuilder, (RexCall) negatedTerm); if (invertNegatedTerm != null) { - negatedTerms.add(invertNegatedTerm); + negatedTerms.add(invertNegatedTerm.toString()); } } break; case IN: - comparedOperands.add(((RexCall) term).operands.get(0)); + comparedOperands.add(((RexCall) term).operands.get(0).toString()); break; case BETWEEN: - comparedOperands.add(((RexCall) term).operands.get(1)); + comparedOperands.add(((RexCall) term).operands.get(1).toString()); break; case IS_NOT_NULL: - notNullOperands.add( - ((RexCall) term).getOperands().get(0)); + notNullOperands.add(((RexCall) term).getOperands().get(0)); terms.remove(i); --i; break; case IS_NULL: - nullOperands.add( - ((RexCall) term).getOperands().get(0)); + nullOperands.add(((RexCall) term).getOperands().get(0).toString()); } } - if (terms.isEmpty() && notTerms.isEmpty() && notNullOperands.isEmpty()) { - return rexBuilder.makeLiteral(true); - } // If one column should be null and is in a comparison predicate, // it is not satisfiable. // Example. IS NULL(x) AND x < 5 - not satisfiable @@ -237,10 +315,9 @@ public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e) { // // Example. IS NOT NULL(x) AND x < 5 : x < 5 for (RexNode operand : notNullOperands) { - if (!comparedOperands.contains(operand)) { + if (!comparedOperands.contains(operand.toString())) { terms.add( - rexBuilder.makeCall( - SqlStdOperatorTable.IS_NOT_NULL, operand)); + rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, operand)); } } // If one of the not-disjunctions is a disjunction that is wholly @@ -253,23 +330,24 @@ public static RexNode simplifyAnd(RexBuilder rexBuilder, RexCall e) { final Set termsSet = new HashSet( Lists.transform(terms, HiveCalciteUtil.REX_STR_FN)); for (RexNode notDisjunction : notTerms) { - final Set notSet = new HashSet( - Lists.transform(RelOptUtil.conjunctions(notDisjunction), HiveCalciteUtil.REX_STR_FN)); - if (termsSet.containsAll(notSet)) { + if (!HiveCalciteUtil.isDeterministic(notDisjunction)) { + continue; + } + final List terms2Set = Lists.transform( + RelOptUtil.conjunctions(notDisjunction), HiveCalciteUtil.REX_STR_FN); + if (termsSet.containsAll(terms2Set)) { return rexBuilder.makeLiteral(false); } } // Add the NOT disjunctions back in. for (RexNode notDisjunction : notTerms) { terms.add( - rexBuilder.makeCall( - SqlStdOperatorTable.NOT, notDisjunction)); + simplify(rexBuilder, + rexBuilder.makeCall(SqlStdOperatorTable.NOT, notDisjunction), true)); } - // The negated terms - for (RexNode notDisjunction : negatedTerms) { - final Set notSet = new HashSet( - Lists.transform(RelOptUtil.conjunctions(notDisjunction), HiveCalciteUtil.REX_STR_FN)); - if (termsSet.containsAll(notSet)) { + // The negated terms: only deterministic expressions + for (String negatedTerm : negatedTerms) { + if (termsSet.contains(negatedTerm)) { return rexBuilder.makeLiteral(false); } } @@ -284,11 +362,13 @@ public static RexNode simplifyOr(RexBuilder rexBuilder, RexCall call) { final RexNode term = terms.get(i); switch (term.getKind()) { case LITERAL: - if (RexLiteral.booleanValue(term)) { - return term; // true - } else { - terms.remove(i); - --i; + if (!RexLiteral.isNullLiteral(term)) { + if (RexLiteral.booleanValue(term)) { + return term; // true + } else { + terms.remove(i); + --i; + } } } } @@ -330,4 +410,78 @@ private static RexCall invert(RexBuilder rexBuilder, RexCall call) { } return null; } + + private static SqlOperator op(SqlKind kind) { + switch (kind) { + case IS_FALSE: + return SqlStdOperatorTable.IS_FALSE; + case IS_TRUE: + return SqlStdOperatorTable.IS_TRUE; + case IS_UNKNOWN: + return SqlStdOperatorTable.IS_UNKNOWN; + case IS_NULL: + return SqlStdOperatorTable.IS_NULL; + case IS_NOT_FALSE: + return SqlStdOperatorTable.IS_NOT_FALSE; + case IS_NOT_TRUE: + return SqlStdOperatorTable.IS_NOT_TRUE; + case IS_NOT_NULL: + return SqlStdOperatorTable.IS_NOT_NULL; + default: + throw new AssertionError(kind); + } + } + + public static class ExprSimplifier extends RexShuttle { + private final RexBuilder rexBuilder; + private final Map unknownAsFalseMap; + + public ExprSimplifier( + RexBuilder rexBuilder) { + this.rexBuilder = rexBuilder; + this.unknownAsFalseMap = new HashMap<>(); + } + + @Override public RexNode visitCall(RexCall call) { + Boolean unknownAsFalse; + switch (call.getKind()) { + case AND: + case CASE: + case EQUALS: + case NOT_EQUALS: + case LESS_THAN: + case GREATER_THAN: + case LESS_THAN_OR_EQUAL: + case GREATER_THAN_OR_EQUAL: + unknownAsFalse = this.unknownAsFalseMap.get(call); + if (unknownAsFalse == null) { + unknownAsFalse = true; + } + break; + default: + unknownAsFalse = false; + } + for (RexNode operand : call.operands) { + this.unknownAsFalseMap.put(operand, unknownAsFalse); + } + RexNode node = super.visitCall(call); + RexNode simplifiedNode = visit(node, unknownAsFalse); + if (simplifiedNode != null) { + return simplifiedNode; + } + return node; + } + + private RexNode visit(final RexNode call, boolean unknownAsFalse) { + RexNode simplified = HiveRexUtil.simplify(rexBuilder, call, unknownAsFalse); + if (simplified == null) { + return null; + } + if (simplified.getType().equals(call.getType())) { + return simplified; + } + return rexBuilder.makeCast(call.getType(), simplified, true); + } + } + } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregatePullUpConstantsRule.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregatePullUpConstantsRule.java new file mode 100644 index 0000000..370c0ec --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveAggregatePullUpConstantsRule.java @@ -0,0 +1,47 @@ +/** + * 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.hadoop.hive.ql.optimizer.calcite.rules; + +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate; + +public class HiveAggregatePullUpConstantsRule extends AggregateProjectPullUpConstantsRule { + + public static final HiveAggregatePullUpConstantsRule INSTANCE = + new HiveAggregatePullUpConstantsRule(); + + public HiveAggregatePullUpConstantsRule() { + super(HiveAggregate.class, RelNode.class, + HiveRelFactories.HIVE_BUILDER, "HiveAggregatePullUpConstantsRule"); + } + + @Override + public boolean matches(RelOptRuleCall call) { + final Aggregate aggregate = call.rel(0); + // Rule cannot be applied if there are GroupingSets + if (aggregate.indicator) { + return false; + } + return super.matches(call); + } + +} diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectFilterPullUpConstantsRule.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectFilterPullUpConstantsRule.java new file mode 100644 index 0000000..bec172b --- /dev/null +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectFilterPullUpConstantsRule.java @@ -0,0 +1,170 @@ +/** + * 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.hadoop.hive.ql.optimizer.calcite.rules; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter; +import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject; + +import com.google.common.collect.Lists; + +/** + * Planner rule that infers constant expressions from Filter into + * a Project operator. + */ +public class HiveProjectFilterPullUpConstantsRule extends RelOptRule { + + public static final HiveProjectFilterPullUpConstantsRule INSTANCE = + new HiveProjectFilterPullUpConstantsRule(HiveProject.class, HiveFilter.class, + HiveRelFactories.HIVE_BUILDER); + + public HiveProjectFilterPullUpConstantsRule( + Class projectClass, + Class filterClass, + RelBuilderFactory relBuilderFactory) { + super(operand(projectClass, + operand(filterClass, any())), + relBuilderFactory, null); + } + + @Override + public boolean matches(RelOptRuleCall call) { + final Filter filterRel = call.rel(1); + RexNode condition = filterRel.getCondition(); + if (!HiveCalciteUtil.isDeterministic(condition)) { + return false; + } + + return super.matches(call); + } + + public void onMatch(RelOptRuleCall call) { + final Project project = call.rel(0); + final Filter filter = call.rel(1); + final RelBuilder builder = call.builder(); + + List projects = project.getChildExps(); + List newProjects = rewriteProjects(projects, filter.getCondition(), builder); + if (newProjects == null) { + return; + } + RelNode newProjRel = builder.push(filter) + .project(newProjects, project.getRowType().getFieldNames()).build(); + call.transformTo(newProjRel); + } + + // Rewrite projects to replace column references by constants when possible + @SuppressWarnings("incomplete-switch") + private static List rewriteProjects(List projects, + RexNode newPushedCondition, RelBuilder relBuilder) { + final List conjunctions = RelOptUtil.conjunctions(newPushedCondition); + final Map conditions = new HashMap(); + for (RexNode conjunction: conjunctions) { + // 1.1. If it is not a RexCall, we continue + if (!(conjunction instanceof RexCall)) { + continue; + } + // 1.2. We extract the information that we need + RexCall conjCall = (RexCall) conjunction; + switch (conjCall.getOperator().getKind()) { + case EQUALS: + if (!(RexUtil.isConstant(conjCall.operands.get(0))) && + RexUtil.isConstant(conjCall.operands.get(1))) { + conditions.put(conjCall.operands.get(0).toString(), conjCall.operands.get(1)); + } else if (!(RexUtil.isConstant(conjCall.operands.get(1))) && + RexUtil.isConstant(conjCall.operands.get(0))) { + conditions.put(conjCall.operands.get(1).toString(), conjCall.operands.get(0)); + } + break; + case IS_NULL: + conditions.put(conjCall.operands.get(0).toString(), relBuilder.getRexBuilder().makeNullLiteral( + conjCall.operands.get(0).getType().getSqlTypeName())); + } + } + + RexReplacer replacer = new RexReplacer(relBuilder.getRexBuilder(), conditions); + List newProjects = Lists.newArrayList(projects); + replacer.mutate(newProjects); + if (replacer.replaced) { + return newProjects; + } + return null; + } + + protected static class RexReplacer extends RexShuttle { + private final RexBuilder rexBuilder; + private final Map replacements; + private boolean replaced; + + RexReplacer( + RexBuilder rexBuilder, + Map replacements) { + this.rexBuilder = rexBuilder; + this.replacements = replacements; + this.replaced = false; + } + + @Override public RexNode visitInputRef(RexInputRef inputRef) { + RexNode node = visit(inputRef); + if (node == null) { + return super.visitInputRef(inputRef); + } + this.replaced = true; + return node; + } + + @Override public RexNode visitCall(RexCall call) { + RexNode node = visit(call); + if (node != null) { + this.replaced = true; + return node; + } + return super.visitCall(call); + } + + private RexNode visit(final RexNode call) { + RexNode replacement = replacements.get(call.toString()); + if (replacement == null) { + return null; + } + if (replacement.getType().equals(call.getType())) { + return replacement; + } + return rexBuilder.makeCast(call.getType(), replacement, true); + } + } +} diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java index 9006f45..3048fcb 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveReduceExpressionsRule.java @@ -61,6 +61,7 @@ import org.apache.calcite.util.Util; import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRelFactories; import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil; +import org.apache.hadoop.hive.ql.optimizer.calcite.HiveRexUtil.ExprSimplifier; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveJoin; import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject; @@ -127,8 +128,11 @@ public FilterReduceExpressionsRule(Class filterClass, final Filter filter = call.rel(0); final RexBuilder rexBuilder = filter.getCluster().getRexBuilder(); - RexNode newConditionExp = HiveRexUtil.simplify(rexBuilder, filter.getCondition()); - final List expList = Lists.newArrayList(newConditionExp); + ExprSimplifier simplifier = new ExprSimplifier(rexBuilder); + List expList = Lists.newArrayList(filter.getCondition()); + simplifier.mutate(expList); + + RexNode newConditionExp = expList.get(0); boolean reduced = false; final RelOptPredicateList predicates = RelMetadataQuery.instance().getPulledUpPredicates(filter.getInput()); diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java index dccd1d9..f166bb6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/JoinTypeCheckCtx.java @@ -53,7 +53,7 @@ public JoinTypeCheckCtx(RowResolver leftRR, RowResolver rightRR, JoinType hiveJoinType) throws SemanticException { - super(RowResolver.getCombinedRR(leftRR, rightRR), true, false, false, false, false, false, false, + super(RowResolver.getCombinedRR(leftRR, rightRR), true, false, false, false, false, false, false, false, false, false); this.inputRRLst = ImmutableList.of(leftRR, rightRR); this.outerJoin = (hiveJoinType == JoinType.LEFTOUTER) || (hiveJoinType == JoinType.RIGHTOUTER) diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java index ee4f4ea..4ed4c6e 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/RexNodeConverter.java @@ -74,7 +74,9 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToChar; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDate; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDecimal; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToUnixTimeStamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFToVarchar; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUnixTimeStamp; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFWhen; import org.apache.hadoop.hive.serde2.objectinspector.ConstantObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; @@ -157,7 +159,7 @@ private RexNode convert(final ExprNodeFieldDesc fieldDesc) throws SemanticExcept } } - private RexNode convert(final ExprNodeGenericFuncDesc func) throws SemanticException { + private RexNode convert(ExprNodeGenericFuncDesc func) throws SemanticException { ExprNodeDesc tmpExprNode; RexNode tmpRN; @@ -174,6 +176,8 @@ private RexNode convert(final ExprNodeGenericFuncDesc func) throws SemanticExcep ((PrimitiveTypeInfo) func.getTypeInfo()).getPrimitiveCategory()))); boolean isCompare = !isNumeric && tgtUdf instanceof GenericUDFBaseCompare; boolean isWhenCase = tgtUdf instanceof GenericUDFWhen || tgtUdf instanceof GenericUDFCase; + boolean isTransformableTimeStamp = func.getGenericUDF() instanceof GenericUDFUnixTimeStamp && + func.getChildren().size() != 0; if (isNumeric) { tgtDT = func.getTypeInfo(); @@ -189,6 +193,9 @@ private RexNode convert(final ExprNodeGenericFuncDesc func) throws SemanticExcep if (checkForStatefulFunctions(func.getChildren())) { throw new SemanticException("Stateful expressions cannot be used inside of CASE"); } + } else if (isTransformableTimeStamp) { + // unix_timestamp(args) -> to_unix_timestamp(args) + func = ExprNodeGenericFuncDesc.newInstance(new GenericUDFToUnixTimeStamp(), func.getChildren()); } for (ExprNodeDesc childExpr : func.getChildren()) { diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java index 0b76bff..3ca1cd9 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java @@ -204,6 +204,7 @@ public static ASTNode buildAST(SqlOperator op, List children) { case BETWEEN: case ROW: case IS_NOT_NULL: + case IS_NULL: case CASE: node = (ASTNode) ParseDriver.adaptor.create(HiveParser.TOK_FUNCTION, "TOK_FUNCTION"); node.addChild((ASTNode) ParseDriver.adaptor.create(hToken.type, hToken.text)); @@ -329,6 +330,7 @@ private static String getName(GenericUDF hiveUDF) { registerFunction("between", HiveBetween.INSTANCE, hToken(HiveParser.Identifier, "between")); registerFunction("struct", SqlStdOperatorTable.ROW, hToken(HiveParser.Identifier, "struct")); registerFunction("isnotnull", SqlStdOperatorTable.IS_NOT_NULL, hToken(HiveParser.TOK_ISNOTNULL, "TOK_ISNOTNULL")); + registerFunction("isnull", SqlStdOperatorTable.IS_NULL, hToken(HiveParser.TOK_ISNULL, "TOK_ISNULL")); registerFunction("when", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when")); registerDuplicateFunction("case", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when")); } diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java index 2825f77..1ab33c6 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/TypeConverter.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hive.ql.optimizer.calcite.translator; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.LinkedList; import java.util.List; @@ -29,14 +30,19 @@ import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.SqlCollation; import org.apache.calcite.sql.SqlIntervalQualifier; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.util.ConversionUtil; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.common.type.HiveChar; import org.apache.hadoop.hive.common.type.HiveVarchar; import org.apache.hadoop.hive.ql.exec.ColumnInfo; import org.apache.hadoop.hive.ql.exec.RowSchema; import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException; import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException.UnsupportedFeature; +import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveReduceExpressionsRule; import org.apache.hadoop.hive.ql.optimizer.calcite.translator.SqlFunctionConverter.HiveToken; import org.apache.hadoop.hive.ql.parse.HiveParser; import org.apache.hadoop.hive.ql.parse.RowResolver; @@ -57,6 +63,9 @@ import com.google.common.collect.Lists; public class TypeConverter { + + protected static final Log LOG = LogFactory.getLog(TypeConverter.class); + private static final Map calciteToHiveTypeNameMap; // TODO: Handling of char[], varchar[], string... @@ -162,7 +171,9 @@ public static RelDataType convert(PrimitiveTypeInfo type, RelDataTypeFactory dtF convertedType = dtFactory.createSqlType(SqlTypeName.DOUBLE); break; case STRING: - convertedType = dtFactory.createSqlType(SqlTypeName.VARCHAR, Integer.MAX_VALUE); + convertedType = dtFactory.createTypeWithCharsetAndCollation( + dtFactory.createSqlType(SqlTypeName.VARCHAR, Integer.MAX_VALUE), + Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME), SqlCollation.IMPLICIT); break; case DATE: convertedType = dtFactory.createSqlType(SqlTypeName.DATE); @@ -187,12 +198,14 @@ public static RelDataType convert(PrimitiveTypeInfo type, RelDataTypeFactory dtF .createSqlType(SqlTypeName.DECIMAL, dtInf.precision(), dtInf.scale()); break; case VARCHAR: - convertedType = dtFactory.createSqlType(SqlTypeName.VARCHAR, - ((BaseCharTypeInfo) type).getLength()); + convertedType = dtFactory.createTypeWithCharsetAndCollation( + dtFactory.createSqlType(SqlTypeName.VARCHAR, ((BaseCharTypeInfo) type).getLength()), + Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME), SqlCollation.IMPLICIT); break; case CHAR: - convertedType = dtFactory.createSqlType(SqlTypeName.CHAR, - ((BaseCharTypeInfo) type).getLength()); + convertedType = dtFactory.createTypeWithCharsetAndCollation( + dtFactory.createSqlType(SqlTypeName.CHAR, ((BaseCharTypeInfo) type).getLength()), + Charset.forName(ConversionUtil.NATIVE_UTF16_CHARSET_NAME), SqlCollation.IMPLICIT); break; case UNKNOWN: convertedType = dtFactory.createSqlType(SqlTypeName.OTHER); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java index 8e00e0b..1ab323d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java @@ -140,6 +140,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveUnion; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAggregateJoinTransposeRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAggregateProjectMergeRule; +import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveAggregatePullUpConstantsRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveExpandDistinctAggregatesRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterAggregateTransposeRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterJoinRule; @@ -156,6 +157,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePointLookupOptimizerRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule; +import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectFilterPullUpConstantsRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectMergeRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectSortTransposeRule; import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveReduceExpressionsRule; @@ -1163,6 +1165,8 @@ private RelNode applyPreJoinOrderingTransforms(RelNode basePlan, RelMetadataProv rules.add(HiveJoinAddNotNullRule.INSTANCE_SEMIJOIN); rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_JOIN); rules.add(HiveJoinPushTransitivePredicatesRule.INSTANCE_SEMIJOIN); + rules.add(HiveAggregatePullUpConstantsRule.INSTANCE); + rules.add(HiveProjectFilterPullUpConstantsRule.INSTANCE); perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER); basePlan = hepPlan(basePlan, true, mdProvider, executorProvider, HepMatchOrder.BOTTOM_UP, rules.toArray(new RelOptRule[rules.size()])); @@ -1214,8 +1218,8 @@ private RelNode applyPreJoinOrderingTransforms(RelNode basePlan, RelMetadataProv // 8. Merge Project-Project if possible perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER); - basePlan = hepPlan(basePlan, false, mdProvider, null, new ProjectMergeRule(true, - HiveRelFactories.HIVE_PROJECT_FACTORY)); + basePlan = hepPlan(basePlan, false, mdProvider, null, + HiveProjectMergeRule.INSTANCE, ProjectRemoveRule.INSTANCE); perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Prejoin ordering transformation, Merge Project-Project"); diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java index cfe4497..9bc835d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java @@ -35,8 +35,8 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Queue; import java.util.Map.Entry; +import java.util.Queue; import java.util.Set; import java.util.TreeSet; import java.util.UUID; @@ -71,7 +71,6 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey; import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; import org.apache.hadoop.hive.ql.CompilationOpContext; -import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.ErrorMsg; import org.apache.hadoop.hive.ql.QueryProperties; import org.apache.hadoop.hive.ql.QueryState; @@ -194,7 +193,6 @@ import org.apache.hadoop.hive.ql.plan.ptf.PTFExpressionDef; import org.apache.hadoop.hive.ql.plan.ptf.PartitionedTableFunctionDef; import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.ResourceType; import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; @@ -3142,8 +3140,18 @@ private Operator genFilterPlan(QB qb, ASTNode condn, Operator input, boolean use OpParseContext inputCtx = opParseCtx.get(input); RowResolver inputRR = inputCtx.getRowResolver(); + + ExprNodeDesc filterCond = genExprNodeDesc(condn, inputRR, useCaching, isCBOExecuted()); + // If filter condition is TRUE, we ignore it + if (filterCond instanceof ExprNodeConstantDesc) { + ExprNodeConstantDesc c = (ExprNodeConstantDesc) filterCond; + if (Boolean.TRUE.equals(c.getValue())) { + return input; + } + } + Operator output = putOpInsertMap(OperatorFactory.getAndMakeChild( - new FilterDesc(genExprNodeDesc(condn, inputRR, useCaching), false), new RowSchema( + new FilterDesc(filterCond, false), new RowSchema( inputRR.getColumnInfos()), input), inputRR); if (LOG.isDebugEnabled()) { @@ -4146,7 +4154,7 @@ static boolean isRegex(String pattern, HiveConf conf) { expr, col_list, null, inputRR, starRR, pos, out_rwsch, qb.getAliases(), false); } else { // Case when this is an expression - TypeCheckCtx tcCtx = new TypeCheckCtx(inputRR); + TypeCheckCtx tcCtx = new TypeCheckCtx(inputRR, true, isCBOExecuted()); // We allow stateful functions in the SELECT list (but nowhere else) tcCtx.setAllowStatefulFunctions(true); tcCtx.setAllowDistinctFunctions(false); @@ -7778,7 +7786,7 @@ private Operator genJoinOperatorChildren(QBJoinTree join, Operator left, List expressions = joinTree.getExpressions().get(i); joinKeys[i] = new ExprNodeDesc[expressions.size()]; for (int j = 0; j < joinKeys[i].length; j++) { - joinKeys[i][j] = genExprNodeDesc(expressions.get(j), inputRR); + joinKeys[i][j] = genExprNodeDesc(expressions.get(j), inputRR, true, isCBOExecuted()); } } // Type checking and implicit type conversion for join keys @@ -11000,12 +11008,18 @@ public ExprNodeDesc genExprNodeDesc(ASTNode expr, RowResolver input) throws SemanticException { // Since the user didn't supply a customized type-checking context, // use default settings. - return genExprNodeDesc(expr, input, true); + return genExprNodeDesc(expr, input, true, false); } public ExprNodeDesc genExprNodeDesc(ASTNode expr, RowResolver input, boolean useCaching) throws SemanticException { - TypeCheckCtx tcCtx = new TypeCheckCtx(input, useCaching); + return genExprNodeDesc(expr, input, useCaching, false); + } + + public ExprNodeDesc genExprNodeDesc(ASTNode expr, RowResolver input, boolean useCaching, + boolean foldExpr) + throws SemanticException { + TypeCheckCtx tcCtx = new TypeCheckCtx(input, useCaching, foldExpr); return genExprNodeDesc(expr, input, tcCtx); } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java index de1c043..02896ff 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckCtx.java @@ -37,6 +37,8 @@ private final boolean useCaching; + private final boolean foldExpr; + /** * Receives translations which will need to be applied during unparse. */ @@ -79,20 +81,21 @@ * The input row resolver of the previous operator. */ public TypeCheckCtx(RowResolver inputRR) { - this(inputRR, true); + this(inputRR, true, false); } - public TypeCheckCtx(RowResolver inputRR, boolean useCaching) { - this(inputRR, useCaching, false, true, true, true, true, true, true, true); + public TypeCheckCtx(RowResolver inputRR, boolean useCaching, boolean foldExpr) { + this(inputRR, useCaching, foldExpr, false, true, true, true, true, true, true, true); } - public TypeCheckCtx(RowResolver inputRR, boolean useCaching, boolean allowStatefulFunctions, - boolean allowDistinctFunctions, boolean allowGBExprElimination, boolean allowAllColRef, - boolean allowFunctionStar, boolean allowWindowing, + public TypeCheckCtx(RowResolver inputRR, boolean useCaching, boolean foldExpr, + boolean allowStatefulFunctions, boolean allowDistinctFunctions, boolean allowGBExprElimination, + boolean allowAllColRef, boolean allowFunctionStar, boolean allowWindowing, boolean allowIndexExpr, boolean allowSubQueryExpr) { setInputRR(inputRR); error = null; this.useCaching = useCaching; + this.foldExpr = foldExpr; this.allowStatefulFunctions = allowStatefulFunctions; this.allowDistinctFunctions = allowDistinctFunctions; this.allowGBExprElimination = allowGBExprElimination; @@ -209,4 +212,8 @@ public boolean getallowSubQueryExpr() { public boolean isUseCaching() { return useCaching; } + + public boolean isFoldExpr() { + return foldExpr; + } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java index da236d5..ca0416b 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/TypeCheckProcFactory.java @@ -52,10 +52,12 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx; import org.apache.hadoop.hive.ql.lib.Rule; import org.apache.hadoop.hive.ql.lib.RuleRegExp; +import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory; import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeColumnListDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeConstantDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; +import org.apache.hadoop.hive.ql.plan.ExprNodeDescUtils; import org.apache.hadoop.hive.ql.plan.ExprNodeFieldDesc; import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc; import org.apache.hadoop.hive.ql.udf.SettableUDF; @@ -1059,6 +1061,17 @@ protected ExprNodeDesc getXpathOrFuncExprNodeDesc(ASTNode expr, desc = ExprNodeGenericFuncDesc.newInstance(genericUDF, funcText, children); } + + // If the function is deterministic and the children are constants, + // we try to fold the expression to remove e.g. cast on constant + if (ctx.isFoldExpr() && desc instanceof ExprNodeGenericFuncDesc && + FunctionRegistry.isDeterministic(genericUDF) && + ExprNodeDescUtils.isAllConstants(children)) { + ExprNodeDesc constantExpr = ConstantPropagateProcFactory.foldExpr((ExprNodeGenericFuncDesc)desc); + if (constantExpr != null) { + desc = constantExpr; + } + } } // UDFOPPositive is a no-op. // However, we still create it, and then remove it here, to make sure we diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java index c6f8907..2b7b0c3 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/ExprNodeDescUtils.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.FunctionRegistry; import org.apache.hadoop.hive.ql.exec.Operator; import org.apache.hadoop.hive.ql.exec.UDF; +import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcFactory; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; import org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge; @@ -210,15 +211,25 @@ public static boolean isDeterministic(ExprNodeDesc desc) { */ public static ArrayList backtrack(List sources, Operator current, Operator terminal) throws SemanticException { + return backtrack(sources, current, terminal, false); + } + + public static ArrayList backtrack(List sources, + Operator current, Operator terminal, boolean foldExpr) throws SemanticException { ArrayList result = new ArrayList(); for (ExprNodeDesc expr : sources) { - result.add(backtrack(expr, current, terminal)); + result.add(backtrack(expr, current, terminal, foldExpr)); } return result; } public static ExprNodeDesc backtrack(ExprNodeDesc source, Operator current, Operator terminal) throws SemanticException { + return backtrack(source, current, terminal, false); + } + + public static ExprNodeDesc backtrack(ExprNodeDesc source, Operator current, + Operator terminal, boolean foldExpr) throws SemanticException { Operator parent = getSingleParent(current, terminal); if (parent == null) { return source; @@ -226,7 +237,7 @@ public static ExprNodeDesc backtrack(ExprNodeDesc source, Operator current, if (source instanceof ExprNodeGenericFuncDesc) { // all children expression should be resolved ExprNodeGenericFuncDesc function = (ExprNodeGenericFuncDesc) source.clone(); - List children = backtrack(function.getChildren(), current, terminal); + List children = backtrack(function.getChildren(), current, terminal, foldExpr); for (ExprNodeDesc child : children) { if (child == null) { // Could not resolve all of the function children, fail @@ -234,6 +245,13 @@ public static ExprNodeDesc backtrack(ExprNodeDesc source, Operator current, } } function.setChildren(children); + if (foldExpr) { + // fold after replacing, if possible + ExprNodeDesc foldedFunction = ConstantPropagateProcFactory.foldExpr(function); + if (foldedFunction != null) { + return foldedFunction; + } + } return function; } if (source instanceof ExprNodeColumnDesc) { @@ -243,7 +261,7 @@ public static ExprNodeDesc backtrack(ExprNodeDesc source, Operator current, if (source instanceof ExprNodeFieldDesc) { // field expression should be resolved ExprNodeFieldDesc field = (ExprNodeFieldDesc) source.clone(); - ExprNodeDesc fieldDesc = backtrack(field.getDesc(), current, terminal); + ExprNodeDesc fieldDesc = backtrack(field.getDesc(), current, terminal, foldExpr); if (fieldDesc == null) { return null; } @@ -485,6 +503,25 @@ public static void getExprNodeColumnDesc(ExprNodeDesc exprDesc, } } + public static boolean isConstant(ExprNodeDesc value) { + if (value instanceof ExprNodeConstantDesc) { + return true; + } + if (value instanceof ExprNodeGenericFuncDesc) { + ExprNodeGenericFuncDesc func = (ExprNodeGenericFuncDesc) value; + if (!FunctionRegistry.isDeterministic(func.getGenericUDF())) { + return false; + } + for (ExprNodeDesc child : func.getChildren()) { + if (!isConstant(child)) { + return false; + } + } + return true; + } + return false; + } + public static boolean isAllConstants(List value) { for (ExprNodeDesc expr : value) { if (!(expr instanceof ExprNodeConstantDesc)) { @@ -641,4 +678,35 @@ public static ExprNodeColumnDesc getColumnExpr(ExprNodeDesc expr) { } return (expr instanceof ExprNodeColumnDesc) ? (ExprNodeColumnDesc)expr : null; } + + // Find the constant origin of a certain column if it is originated from a constant + // Otherwise, it returns the expression that originated the column + public static ExprNodeDesc findConstantExprOrigin(String dpCol, Operator op) { + ExprNodeDesc expr = op.getColumnExprMap().get(dpCol); + ExprNodeDesc foldedExpr; + // If it is a function, we try to fold it + if (expr instanceof ExprNodeGenericFuncDesc) { + foldedExpr = ConstantPropagateProcFactory.foldExpr((ExprNodeGenericFuncDesc)expr); + if (foldedExpr == null) { + foldedExpr = expr; + } + } else { + foldedExpr = expr; + } + // If it is a column reference, we will try to resolve it + if (foldedExpr instanceof ExprNodeColumnDesc) { + Operator originOp = null; + for(Operator parentOp : op.getParentOperators()) { + if (parentOp.getColumnExprMap() != null) { + originOp = parentOp; + break; + } + } + if (originOp != null) { + return findConstantExprOrigin(((ExprNodeColumnDesc)foldedExpr).getColumn(), originOp); + } + } + // Otherwise, we return the expression + return foldedExpr; + } } diff --git ql/src/test/queries/clientpositive/join_view.q ql/src/test/queries/clientpositive/join_view.q index 16b6816..69c96be 100644 --- ql/src/test/queries/clientpositive/join_view.q +++ ql/src/test/queries/clientpositive/join_view.q @@ -3,8 +3,6 @@ drop table invites2; create table invites (foo int, bar string) partitioned by (ds string); create table invites2 (foo int, bar string) partitioned by (ds string); -set hive.mapred.mode=strict; - -- test join views: see HIVE-1989 create view v as select invites.bar, invites2.foo, invites2.ds from invites join invites2 on invites.ds=invites2.ds; @@ -13,4 +11,4 @@ explain select * from v where ds='2011-09-01'; drop view v; drop table invites; -drop table invites2; \ No newline at end of file +drop table invites2; diff --git ql/src/test/results/clientpositive/annotate_stats_filter.q.out ql/src/test/results/clientpositive/annotate_stats_filter.q.out index ba0419e..bded288 100644 --- ql/src/test/results/clientpositive/annotate_stats_filter.q.out +++ ql/src/test/results/clientpositive/annotate_stats_filter.q.out @@ -262,7 +262,7 @@ STAGE PLANS: predicate: zip is null (type: boolean) Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: state (type: string), locid (type: int), null (type: bigint), year (type: int) + expressions: state (type: string), locid (type: int), null (type: void), year (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -297,15 +297,15 @@ STAGE PLANS: alias: loc_orc Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: zip is null (type: boolean) + predicate: (not zip is not null) (type: boolean) Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: state (type: string), locid (type: int), null (type: bigint), year (type: int) + expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -375,7 +375,7 @@ STAGE PLANS: alias: loc_orc Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: zip is not null (type: boolean) + predicate: (not zip is null) (type: boolean) Statistics: Num rows: 7 Data size: 702 Basic stats: COMPLETE Column stats: COMPLETE Select Operator expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int) @@ -718,15 +718,15 @@ STAGE PLANS: alias: loc_orc Statistics: Num rows: 8 Data size: 804 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator - predicate: ((year = 2001) and year is null) (type: boolean) + predicate: false (type: boolean) Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: state (type: string), locid (type: int), zip (type: bigint), null (type: int) + expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: int) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 98 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 1 Data size: 102 Basic stats: COMPLETE Column stats: COMPLETE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git ql/src/test/results/clientpositive/archive_excludeHadoop20.q.out ql/src/test/results/clientpositive/archive_excludeHadoop20.q.out index c2b9872..52d17b4 100644 --- ql/src/test/results/clientpositive/archive_excludeHadoop20.q.out +++ ql/src/test/results/clientpositive/archive_excludeHadoop20.q.out @@ -137,6 +137,7 @@ POSTHOOK: Input: default@tstsrcpart POSTHOOK: Input: default@tstsrcpart@ds=2008-04-08/hr=12 #### A masked pattern was here #### 0 3 +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: SELECT * FROM tstsrcpart a JOIN tstsrc b ON a.key=b.key WHERE a.ds='2008-04-08' AND a.hr='12' AND a.key='0' PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/archive_multi.q.out ql/src/test/results/clientpositive/archive_multi.q.out index 0ad29d1..38f3f1a 100644 --- ql/src/test/results/clientpositive/archive_multi.q.out +++ ql/src/test/results/clientpositive/archive_multi.q.out @@ -141,6 +141,7 @@ POSTHOOK: Input: ac_test@tstsrcpart POSTHOOK: Input: ac_test@tstsrcpart@ds=2008-04-08/hr=12 #### A masked pattern was here #### 0 3 +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: SELECT * FROM ac_test.tstsrcpart a JOIN ac_test.tstsrc b ON a.key=b.key WHERE a.ds='2008-04-08' AND a.hr='12' AND a.key='0' PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/auto_join8.q.out ql/src/test/results/clientpositive/auto_join8.q.out index 324f95d..8daa1c5 100644 --- ql/src/test/results/clientpositive/auto_join8.q.out +++ ql/src/test/results/clientpositive/auto_join8.q.out @@ -152,7 +152,7 @@ POSTHOOK: Input: default@src POSTHOOK: Output: default@dest1 POSTHOOK: Lineage: dest1.c1 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: dest1.c2 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] -POSTHOOK: Lineage: dest1.c3 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.c3 EXPRESSION [] POSTHOOK: Lineage: dest1.c4 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: SELECT sum(hash(dest1.c1,dest1.c2,dest1.c3,dest1.c4)) FROM dest1 PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/bucket_groupby.q.out ql/src/test/results/clientpositive/bucket_groupby.q.out index d1414fe..232d999 100644 --- ql/src/test/results/clientpositive/bucket_groupby.q.out +++ ql/src/test/results/clientpositive/bucket_groupby.q.out @@ -393,38 +393,34 @@ STAGE PLANS: Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 3 (type: int) + keys: _col0 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), 3 (type: int) - sort order: ++ - Map-reduce partition columns: _col0 (type: string), 3 (type: int) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - value expressions: _col2 (type: bigint) + value expressions: _col1 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 3 (type: int) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col2 (type: bigint) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Limit - Number of rows: 10 + Limit + Number of rows: 10 + Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false Statistics: Num rows: 10 Data size: 100 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 10 Data size: 100 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 + 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 @@ -930,34 +926,30 @@ STAGE PLANS: Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 3 (type: int) + keys: _col0 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), 3 (type: int) - sort order: ++ - Map-reduce partition columns: _col0 (type: string), 3 (type: int) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: bigint) + value expressions: _col1 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 3 (type: int) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col2 (type: bigint) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 500 Data size: 5312 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 + 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 diff --git ql/src/test/results/clientpositive/cbo_rp_join1.q.out ql/src/test/results/clientpositive/cbo_rp_join1.q.out index f3982b8..01a367e 100644 --- ql/src/test/results/clientpositive/cbo_rp_join1.q.out +++ ql/src/test/results/clientpositive/cbo_rp_join1.q.out @@ -141,7 +141,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 2 Data size: 22 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((_col1 = 40) and (_col0 = 40)) (type: boolean) + predicate: ((_col0 = _col1) and (_col1 = 40) and (_col0 = 40)) (type: boolean) Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator sort order: diff --git ql/src/test/results/clientpositive/constantPropWhen.q.out ql/src/test/results/clientpositive/constantPropWhen.q.out index 650cbc7..9d4018b 100644 --- ql/src/test/results/clientpositive/constantPropWhen.q.out +++ ql/src/test/results/clientpositive/constantPropWhen.q.out @@ -36,7 +36,7 @@ STAGE PLANS: alias: test_1 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: (not NVL((id = id2),false)) (type: boolean) + expressions: CASE WHEN ((id = id2)) THEN (false) ELSE (true) END (type: boolean) outputColumnNames: _col0 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -81,7 +81,7 @@ STAGE PLANS: alias: test_1 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: (not NVL((id = id2),false)) (type: boolean) + expressions: CASE WHEN ((id = id2)) THEN (false) ELSE (true) END (type: boolean) outputColumnNames: _col0 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -126,7 +126,7 @@ STAGE PLANS: alias: test_1 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: NVL((id = id2),false) (type: boolean) + expressions: CASE WHEN ((id = id2)) THEN (true) ELSE (false) END (type: boolean) outputColumnNames: _col0 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -171,7 +171,7 @@ STAGE PLANS: alias: test_1 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: NVL((id = id2),false) (type: boolean) + expressions: CASE WHEN ((id = id2)) THEN (true) ELSE (false) END (type: boolean) outputColumnNames: _col0 Statistics: Num rows: 4 Data size: 24 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/constprog_semijoin.q.out ql/src/test/results/clientpositive/constprog_semijoin.q.out index 35d062d..7a084e9 100644 --- ql/src/test/results/clientpositive/constprog_semijoin.q.out +++ ql/src/test/results/clientpositive/constprog_semijoin.q.out @@ -440,7 +440,7 @@ STAGE PLANS: predicate: (((id = 100) = true) and (id <> 100) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: id (type: int), (id = 100) (type: boolean) + expressions: id (type: int), true (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -521,7 +521,7 @@ STAGE PLANS: predicate: ((id) IN (100, 200) and ((id = 100) = true) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: id (type: int), (id = 100) (type: boolean) + expressions: id (type: int), true (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -604,23 +604,25 @@ STAGE PLANS: predicate: (((id = 100) = true) and (id = 200) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: 200 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: 200 (type: int), false (type: boolean) + keys: _col0 (type: int), _col1 (type: boolean) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 200 (type: int), false (type: boolean) + key expressions: _col0 (type: int), _col1 (type: boolean) sort order: ++ - Map-reduce partition columns: 200 (type: int), false (type: boolean) + Map-reduce partition columns: _col0 (type: int), _col1 (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Join Operator condition map: Left Semi Join 0 to 1 keys: - 0 _col3 (type: int), true (type: boolean) + 0 200 (type: int), true (type: boolean) 1 _col0 (type: int), _col1 (type: boolean) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2 Data size: 44 Basic stats: COMPLETE Column stats: NONE @@ -683,23 +685,25 @@ STAGE PLANS: predicate: (((id = 100) = true) and (id = 100) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: 100 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: 100 (type: int), true (type: boolean) + keys: _col0 (type: int), _col1 (type: boolean) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 100 (type: int), true (type: boolean) + key expressions: _col0 (type: int), _col1 (type: boolean) sort order: ++ - Map-reduce partition columns: 100 (type: int), true (type: boolean) + Map-reduce partition columns: _col0 (type: int), _col1 (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Join Operator condition map: Left Semi Join 0 to 1 keys: - 0 _col3 (type: int), true (type: boolean) + 0 100 (type: int), true (type: boolean) 1 _col0 (type: int), _col1 (type: boolean) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2 Data size: 44 Basic stats: COMPLETE Column stats: NONE @@ -764,7 +768,7 @@ STAGE PLANS: predicate: (((id = 100) = true) and id is not null and (id = 100) is not null) (type: boolean) Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: id (type: int), (id = 100) (type: boolean) + expressions: id (type: int), true (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Group By Operator diff --git ql/src/test/results/clientpositive/cp_sel.q.out ql/src/test/results/clientpositive/cp_sel.q.out index f42ec84..9ee9754 100644 --- ql/src/test/results/clientpositive/cp_sel.q.out +++ ql/src/test/results/clientpositive/cp_sel.q.out @@ -16,18 +16,18 @@ STAGE PLANS: alias: srcpart Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string), value (type: string), 'hello' (type: string), 'world' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: 1 (type: int) sort order: + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - value expressions: _col0 (type: string), _col1 (type: string) + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), 'hello' (type: string), 'world' (type: string) + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Limit @@ -87,18 +87,18 @@ STAGE PLANS: alias: srcpart Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string), value (type: string), 'hello' (type: string), 'world' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string) Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), 'hello' (type: string), 'world' (type: string) + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/cte_5.q.out ql/src/test/results/clientpositive/cte_5.q.out index 96a6543..e9d700d 100644 --- ql/src/test/results/clientpositive/cte_5.q.out +++ ql/src/test/results/clientpositive/cte_5.q.out @@ -118,7 +118,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble('5') (type: double) + 1 5.0 (type: double) outputColumnNames: _col0 Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/cte_mat_1.q.out ql/src/test/results/clientpositive/cte_mat_1.q.out index b7d5f7d..bd860ef 100644 --- ql/src/test/results/clientpositive/cte_mat_1.q.out +++ ql/src/test/results/clientpositive/cte_mat_1.q.out @@ -51,18 +51,15 @@ STAGE PLANS: keys: 0 '5' (type: string) 1 '5' (type: string) + outputColumnNames: _col0 Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '5' (type: string) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 275 Data size: 2921 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 + 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 diff --git ql/src/test/results/clientpositive/cte_mat_2.q.out ql/src/test/results/clientpositive/cte_mat_2.q.out index b7d5f7d..bd860ef 100644 --- ql/src/test/results/clientpositive/cte_mat_2.q.out +++ ql/src/test/results/clientpositive/cte_mat_2.q.out @@ -51,18 +51,15 @@ STAGE PLANS: keys: 0 '5' (type: string) 1 '5' (type: string) + outputColumnNames: _col0 Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '5' (type: string) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 275 Data size: 2921 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 + 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 diff --git ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out index 9a09c4c..b43e310 100644 --- ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out +++ ql/src/test/results/clientpositive/dynamic_rdd_cache.q.out @@ -1046,14 +1046,14 @@ STAGE PLANS: outputColumnNames: _col2, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col6 (type: string), _col5 (type: int), _col4 (type: int), _col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col2 (type: int) + outputColumnNames: _col4, _col5, _col6, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator - aggregations: avg(_col4), stddev_samp(_col4) - keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), 3 (type: int) + aggregations: stddev_samp(_col2), avg(_col2) + keys: _col4 (type: int), _col5 (type: int), _col6 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: true @@ -1067,28 +1067,28 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), 3 (type: int) - sort order: ++++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int), 3 (type: int) + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: struct), _col5 (type: struct) + value expressions: _col3 (type: struct), _col4 (type: struct) Reduce Operator Tree: Group By Operator - aggregations: avg(VALUE._col0), stddev_samp(VALUE._col1) - keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: int), 3 (type: int) + aggregations: stddev_samp(VALUE._col0), avg(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), _col5 (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col0 (type: int), _col3 (type: double), _col4 (type: double) + outputColumnNames: _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col5 / _col4) > 1.0)) END (type: boolean) + predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col3 / _col4) > 1.0)) END (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col5 / _col4)) END (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col2 (type: int), 3 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col3 / _col4)) END (type: double) + outputColumnNames: _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: true @@ -1106,14 +1106,14 @@ STAGE PLANS: sort order: ++ Map-reduce partition columns: _col2 (type: int), _col1 (type: int) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: double), _col5 (type: double) + value expressions: _col3 (type: int), _col4 (type: double), _col5 (type: double) TableScan Reduce Output Operator key expressions: _col2 (type: int), _col1 (type: int) sort order: ++ Map-reduce partition columns: _col2 (type: int), _col1 (type: int) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: double), _col5 (type: double) + value expressions: _col3 (type: int), _col4 (type: double), _col5 (type: double) Reduce Operator Tree: Join Operator condition map: @@ -1121,11 +1121,11 @@ STAGE PLANS: keys: 0 _col2 (type: int), _col1 (type: int) 1 _col2 (type: int), _col1 (type: int) - outputColumnNames: _col1, _col2, _col4, _col5, _col7, _col8, _col10, _col11 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), _col5 (type: double), _col7 (type: int), _col8 (type: int), _col10 (type: double), _col11 (type: double) - outputColumnNames: _col0, _col1, _col3, _col4, _col5, _col6, _col8, _col9 + expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: double), _col5 (type: double), _col7 (type: int), _col8 (type: int), _col9 (type: int), _col10 (type: double), _col11 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: true @@ -1139,13 +1139,13 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: int), 3 (type: int), _col3 (type: double), _col4 (type: double), 4 (type: int), _col8 (type: double), _col9 (type: double) + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: double), _col4 (type: double), _col7 (type: int), _col8 (type: double), _col9 (type: double) sort order: ++++++++ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE value expressions: _col5 (type: int), _col6 (type: int) Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), 3 (type: int), KEY.reducesinkkey3 (type: double), KEY.reducesinkkey4 (type: double), VALUE._col0 (type: int), VALUE._col1 (type: int), 4 (type: int), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double) + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: double), KEY.reducesinkkey4 (type: double), VALUE._col0 (type: int), VALUE._col1 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator @@ -1283,14 +1283,14 @@ STAGE PLANS: outputColumnNames: _col2, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col6 (type: string), _col5 (type: int), _col4 (type: int), _col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col2 (type: int) + outputColumnNames: _col4, _col5, _col6, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator - aggregations: avg(_col4), stddev_samp(_col4) - keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), 4 (type: int) + aggregations: stddev_samp(_col2), avg(_col2) + keys: _col4 (type: int), _col5 (type: int), _col6 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: true @@ -1304,28 +1304,28 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), 4 (type: int) - sort order: ++++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int), 4 (type: int) + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: struct), _col5 (type: struct) + value expressions: _col3 (type: struct), _col4 (type: struct) Reduce Operator Tree: Group By Operator - aggregations: avg(VALUE._col0), stddev_samp(VALUE._col1) - keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: int), 4 (type: int) + aggregations: stddev_samp(VALUE._col0), avg(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), _col5 (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col0 (type: int), _col3 (type: double), _col4 (type: double) + outputColumnNames: _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col5 / _col4) > 1.0)) END (type: boolean) + predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col3 / _col4) > 1.0)) END (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col5 / _col4)) END (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col2 (type: int), 4 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col3 / _col4)) END (type: double) + outputColumnNames: _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: true diff --git ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out index 13383fb..165ad02 100644 --- ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out +++ ql/src/test/results/clientpositive/dynpart_sort_optimization.q.out @@ -2289,28 +2289,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (s = 'foo') (type: boolean) - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), t (type: tinyint), i (type: int) - outputColumnNames: _col0, _col1, _col2, _col4, _col5 - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), t (type: tinyint), i (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'foo' (type: string), _col4 (type: tinyint), _col5 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: 'foo' (type: string), _col4 (type: tinyint), _col5 (type: int) - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2354,13 +2354,13 @@ STAGE PLANS: predicate: (t = 27) (type: boolean) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), i (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), 27 (type: tinyint), i (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string), 27 (type: tinyint), _col5 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: _col3 (type: string), 27 (type: tinyint), _col5 (type: int) + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reduce Operator Tree: @@ -2414,13 +2414,13 @@ STAGE PLANS: predicate: (i = 100) (type: boolean) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), t (type: tinyint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4 + expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), t (type: tinyint), 100 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string), _col4 (type: tinyint), 100 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), 100 (type: int) + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reduce Operator Tree: @@ -2474,13 +2474,13 @@ STAGE PLANS: predicate: ((i = 100) and (t = 27)) (type: boolean) Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string) - outputColumnNames: _col0, _col1, _col2, _col3 + expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), 27 (type: tinyint), 100 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string), 27 (type: tinyint), 100 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: _col3 (type: string), 27 (type: tinyint), 100 (type: int) + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reduce Operator Tree: @@ -2529,28 +2529,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((i = 100) and (s = 'foo')) (type: boolean) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), t (type: tinyint) - outputColumnNames: _col0, _col1, _col2, _col4 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), t (type: tinyint), 100 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'foo' (type: string), _col4 (type: tinyint), 100 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: 'foo' (type: string), _col4 (type: tinyint), 100 (type: int) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2589,28 +2589,28 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((t = 27) and (s = 'foo')) (type: boolean) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), i (type: int) - outputColumnNames: _col0, _col1, _col2, _col5 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), 27 (type: tinyint), i (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'foo' (type: string), 27 (type: tinyint), _col5 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: 'foo' (type: string), 27 (type: tinyint), _col5 (type: int) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2654,17 +2654,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((i = 100) and (t = 27) and (s = 'foo')) (type: boolean) - Statistics: Num rows: 493 Data size: 13312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 107 Data size: 13282 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), 27 (type: tinyint), 100 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 493 Data size: 13312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 107 Data size: 13282 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 493 Data size: 13312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 107 Data size: 13282 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out index 3b24a2e..387dfee 100644 --- ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out +++ ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out @@ -1538,29 +1538,29 @@ 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: _col1, _col2 + outputColumnNames: key, value Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(_col2) - keys: 'day' (type: string), _col1 (type: string) + aggregations: count(value) + keys: key (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'day' (type: string), _col1 (type: string) - sort order: ++ - Map-reduce partition columns: 'day' (type: string), _col1 (type: string) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: bigint) + value expressions: _col1 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: 'day' (type: string), KEY._col1 (type: string) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int), 'day' (type: string) + expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int), 'day' (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1654,29 +1654,29 @@ 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: _col1, _col2 + outputColumnNames: key, value Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(_col2) - keys: 'day' (type: string), _col1 (type: string) + aggregations: count(value) + keys: key (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'day' (type: string), _col1 (type: string) - sort order: ++ - Map-reduce partition columns: 'day' (type: string), _col1 (type: string) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: bigint) + value expressions: _col1 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: 'day' (type: string), KEY._col1 (type: string) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int), 'day' (type: string) + expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int), 'day' (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/filter_cond_pushdown.q.out ql/src/test/results/clientpositive/filter_cond_pushdown.q.out index 132b590..76e6665 100644 --- ql/src/test/results/clientpositive/filter_cond_pushdown.q.out +++ ql/src/test/results/clientpositive/filter_cond_pushdown.q.out @@ -302,26 +302,26 @@ STAGE PLANS: predicate: ((c_int = 1) and key is not null) (type: boolean) Statistics: Num rows: 10 Data size: 131 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), c_float (type: float) - outputColumnNames: _col0, _col2 + expressions: key (type: string), 1 (type: int), c_float (type: float) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 10 Data size: 131 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: 1.0 (type: double), _col0 (type: string) sort order: ++ Map-reduce partition columns: 1.0 (type: double), _col0 (type: string) Statistics: Num rows: 10 Data size: 131 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: float) + value expressions: _col1 (type: int), _col2 (type: float) Reduce Operator Tree: Join Operator condition map: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double), _col0 (type: string) - 1 UDFToDouble(1) (type: double), _col0 (type: string) - outputColumnNames: _col0, _col1, _col2, _col5 + 1 1.0 (type: double), _col0 (type: string) + outputColumnNames: _col0, _col1, _col2, _col4, _col5 Statistics: Num rows: 22 Data size: 288 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (((_col2 + _col5) > 2.0) or ((_col1 + 1) > 2)) (type: boolean) + predicate: (((_col2 + _col5) > 2.0) or ((_col1 + _col4) > 2)) (type: boolean) Statistics: Num rows: 14 Data size: 183 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string) diff --git ql/src/test/results/clientpositive/fold_case.q.out ql/src/test/results/clientpositive/fold_case.q.out index 53139da..f19e1b2 100644 --- ql/src/test/results/clientpositive/fold_case.q.out +++ ql/src/test/results/clientpositive/fold_case.q.out @@ -67,10 +67,10 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (key <> '238') (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + predicate: (not (key = '238')) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) mode: hash @@ -267,18 +267,20 @@ STAGE PLANS: TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE - Filter Operator - predicate: false (type: boolean) - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - Group By Operator - aggregations: count(1) - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - Reduce Output Operator - sort order: + Select Operator + Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: null (type: void) + Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE + Group By Operator + aggregations: count(1) + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE - value expressions: _col0 (type: bigint) + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) @@ -368,15 +370,15 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (key <> '238') (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + predicate: (not (key = '238')) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: CASE WHEN ((key = '238')) THEN (null) ELSE (false) END (type: boolean) outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -404,17 +406,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: false (type: boolean) - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + predicate: null (type: void) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: null (type: void) + expressions: CASE WHEN ((key = '238')) THEN (null) ELSE (null) END (type: void) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out index 106ad6b..13f6ab4 100644 --- ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out +++ ql/src/test/results/clientpositive/fold_eq_with_case_when.q.out @@ -173,10 +173,10 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: CASE WHEN ((key <> '238')) THEN ((key = '238')) ELSE ((key = '238')) END (type: boolean) + predicate: (key = '238') (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string) + expressions: '238' (type: string) outputColumnNames: _col0 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/fold_when.q.out ql/src/test/results/clientpositive/fold_when.q.out index 5b68408..b09b8b4 100644 --- ql/src/test/results/clientpositive/fold_when.q.out +++ ql/src/test/results/clientpositive/fold_when.q.out @@ -16,15 +16,15 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: false (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + predicate: null (type: void) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -54,15 +54,15 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: false (type: boolean) - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + predicate: null (type: void) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -156,7 +156,7 @@ STAGE PLANS: predicate: (key = '238') (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string) + expressions: '238' (type: string) outputColumnNames: _col0 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -194,7 +194,7 @@ STAGE PLANS: predicate: (key = '238') (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string) + expressions: '238' (type: string) outputColumnNames: _col0 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -328,15 +328,15 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (key <> '238') (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + predicate: (not (key = '238')) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -435,15 +435,15 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (key <> '11') (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + predicate: (not (key = '11')) (type: boolean) + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git ql/src/test/results/clientpositive/folder_predicate.q.out ql/src/test/results/clientpositive/folder_predicate.q.out index 7fcc172..48a4889 100644 --- ql/src/test/results/clientpositive/folder_predicate.q.out +++ ql/src/test/results/clientpositive/folder_predicate.q.out @@ -37,15 +37,15 @@ STAGE PLANS: alias: predicate_fold_tb Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (value is null or (value <> 3)) (type: boolean) - Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE + predicate: (not (value is not null and (value = 3))) (type: boolean) + Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: int) outputColumnNames: _col0 - Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -88,15 +88,15 @@ STAGE PLANS: alias: predicate_fold_tb Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (value is null or (value < 3)) (type: boolean) - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + predicate: (not (value is not null and (value >= 3))) (type: boolean) + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -137,15 +137,15 @@ STAGE PLANS: alias: predicate_fold_tb Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (value is null or (value > 3)) (type: boolean) - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + predicate: (not (value is not null and (value <= 3))) (type: boolean) + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -186,15 +186,15 @@ STAGE PLANS: alias: predicate_fold_tb Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (value is null or (value <= 3)) (type: boolean) - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + predicate: (not (value is not null and (value > 3))) (type: boolean) + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -236,15 +236,15 @@ STAGE PLANS: alias: predicate_fold_tb Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (value is null or (value >= 3)) (type: boolean) - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + predicate: (not (value is not null and (value < 3))) (type: boolean) + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: int) outputColumnNames: _col0 - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 5 Data size: 5 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 4 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -286,15 +286,15 @@ STAGE PLANS: alias: predicate_fold_tb Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (value is null or (value = 3)) (type: boolean) - Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE + predicate: (not (value is not null and (value <> 3))) (type: boolean) + Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: int) outputColumnNames: _col0 - Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -334,7 +334,7 @@ STAGE PLANS: alias: predicate_fold_tb Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (value is null or (value <= 1) or (value > 3)) (type: boolean) + predicate: (not (value is not null and (value > 1) and (value <= 3))) (type: boolean) Statistics: Num rows: 6 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: int) diff --git ql/src/test/results/clientpositive/groupby_ppd.q.out ql/src/test/results/clientpositive/groupby_ppd.q.out index c63acd3..d411a0d 100644 --- ql/src/test/results/clientpositive/groupby_ppd.q.out +++ ql/src/test/results/clientpositive/groupby_ppd.q.out @@ -28,24 +28,20 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: foo (type: int) - outputColumnNames: _col1 + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 1 (type: int), _col1 (type: int) - outputColumnNames: _col0, _col1 + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Group By Operator - keys: _col0 (type: int), _col1 (type: int) - mode: hash - outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL 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: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE TableScan alias: c Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE @@ -54,32 +50,28 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: foo (type: int) - outputColumnNames: _col1 + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 1 (type: int), _col1 (type: int) - outputColumnNames: _col0, _col1 + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Group By Operator - keys: _col0 (type: int), _col1 (type: int) - mode: hash - outputColumnNames: _col0, _col1 + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL 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: 2 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Operator Tree: Group By Operator - keys: KEY._col0 (type: int), KEY._col1 (type: int) + keys: KEY._col0 (type: int) mode: mergepartial - outputColumnNames: _col0, _col1 + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col0 (type: int) + expressions: _col0 (type: int), 1 (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/groupby_sort_1_23.q.out ql/src/test/results/clientpositive/groupby_sort_1_23.q.out index a6e15ba..4b38a0b 100644 --- ql/src/test/results/clientpositive/groupby_sort_1_23.q.out +++ ql/src/test/results/clientpositive/groupby_sort_1_23.q.out @@ -1337,16 +1337,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string) - outputColumnNames: _col1 + outputColumnNames: key Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: 1 (type: int), _col1 (type: string) + aggregations: count(key) + keys: key (type: string) mode: final - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: 1 (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) + expressions: 1 (type: int), UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1427,7 +1427,7 @@ STAGE PLANS: name: default.t1 name: default.t1 Truncated Path -> Alias: - /t1 [$hdt$_0:t1] + /t1 [t1] Stage: Stage-7 Conditional Operator @@ -1618,7 +1618,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl3 -POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl3.key1 SIMPLE [] POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] PREHOOK: query: SELECT * FROM outputTbl3 @@ -1667,22 +1667,22 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string) - null sort order: aaa - sort order: +++ - Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1736,17 +1736,17 @@ STAGE PLANS: name: default.t1 name: default.t1 Truncated Path -> Alias: - /t1 [$hdt$_0:t1] + /t1 [t1] Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 1 (type: int), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1812,7 +1812,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] @@ -4023,16 +4023,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4118,7 +4118,7 @@ STAGE PLANS: name: default.t2 name: default.t2 Truncated Path -> Alias: - /t2 [$hdt$_0:t2] + /t2 [t2] Stage: Stage-7 Conditional Operator @@ -4344,7 +4344,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -4402,16 +4402,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string), 2 (type: int) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3, _col4 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), 2 (type: int), UDFToInteger(_col4) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), 2 (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4492,7 +4492,7 @@ STAGE PLANS: name: default.t2 name: default.t2 Truncated Path -> Alias: - /t2 [$hdt$_0:t2] + /t2 [t2] Stage: Stage-7 Conditional Operator @@ -4683,7 +4683,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl5 -POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -4738,16 +4738,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -5121,16 +5121,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 2 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out index 38b24a4..79a1577 100644 --- ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out +++ ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out @@ -1403,16 +1403,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string) - outputColumnNames: _col1 + outputColumnNames: key Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: 1 (type: int), _col1 (type: string) + aggregations: count(key) + keys: key (type: string) mode: final - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: 1 (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) + expressions: 1 (type: int), UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1493,7 +1493,7 @@ STAGE PLANS: name: default.t1 name: default.t1 Truncated Path -> Alias: - /t1 [$hdt$_0:t1] + /t1 [t1] Stage: Stage-7 Conditional Operator @@ -1684,7 +1684,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl3 -POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl3.key1 SIMPLE [] POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] PREHOOK: query: SELECT * FROM outputTbl3 @@ -1734,22 +1734,22 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string) - null sort order: aaa - sort order: +++ + key expressions: _col0 (type: string), _col1 (type: string) + null sort order: aa + sort order: ++ Map-reduce partition columns: rand() (type: double) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1803,14 +1803,14 @@ STAGE PLANS: name: default.t1 name: default.t1 Truncated Path -> Alias: - /t1 [$hdt$_0:t1] + /t1 [t1] Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 1 (type: int), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: partials - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -1821,8 +1821,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col2,_col3 - columns.types string,int,string,bigint + columns _col0,_col1,_col2 + columns.types string,string,bigint escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -1836,13 +1836,13 @@ STAGE PLANS: TableScan GatherStats: false Reduce Output Operator - key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string) - null sort order: aaa - sort order: +++ - Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1853,8 +1853,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col2,_col3 - columns.types string,int,string,bigint + columns _col0,_col1,_col2 + columns.types string,string,bigint escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -1862,8 +1862,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col2,_col3 - columns.types string,int,string,bigint + columns _col0,_col1,_col2 + columns.types string,string,bigint escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -1873,12 +1873,12 @@ STAGE PLANS: Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 1 (type: int), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1944,7 +1944,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] @@ -4485,16 +4485,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4580,7 +4580,7 @@ STAGE PLANS: name: default.t2 name: default.t2 Truncated Path -> Alias: - /t2 [$hdt$_0:t2] + /t2 [t2] Stage: Stage-7 Conditional Operator @@ -4806,7 +4806,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -4864,16 +4864,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string), 2 (type: int) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3, _col4 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), 2 (type: int), UDFToInteger(_col4) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), 2 (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4954,7 +4954,7 @@ STAGE PLANS: name: default.t2 name: default.t2 Truncated Path -> Alias: - /t2 [$hdt$_0:t2] + /t2 [t2] Stage: Stage-7 Conditional Operator @@ -5145,7 +5145,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl5 -POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -5200,16 +5200,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -5583,16 +5583,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 2 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/index_auto_unused.q.out ql/src/test/results/clientpositive/index_auto_unused.q.out index 81079f8..6c258b8 100644 --- ql/src/test/results/clientpositive/index_auto_unused.q.out +++ ql/src/test/results/clientpositive/index_auto_unused.q.out @@ -356,25 +356,37 @@ PREHOOK: type: QUERY POSTHOOK: query: EXPLAIN SELECT * FROM srcpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 POSTHOOK: type: QUERY STAGE DEPENDENCIES: - Stage-0 is a root stage + 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: srcpart + filterExpr: ((12.0 = 12.0) and (UDFToDouble(key) < 10.0)) (type: boolean) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: ((12.0 = 12.0) and (UDFToDouble(key) < 10.0)) (type: boolean) + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), '2008-04-09' (type: string), hr (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 166 Data size: 1763 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: - TableScan - alias: srcpart - filterExpr: (UDFToDouble(key) < 10.0) (type: boolean) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: (UDFToDouble(key) < 10.0) (type: boolean) - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: key (type: string), value (type: string), '2008-04-09' (type: string), hr (type: string) - outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE - ListSink + ListSink PREHOOK: query: SELECT * FROM srcpart WHERE ds='2008-04-09' AND hr=12 AND key < 10 PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/input23.q.out ql/src/test/results/clientpositive/input23.q.out index dcb2891..e03c9e7 100644 --- ql/src/test/results/clientpositive/input23.q.out +++ ql/src/test/results/clientpositive/input23.q.out @@ -1,4 +1,4 @@ -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain extended select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5 PREHOOK: type: QUERY @@ -144,7 +144,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select * from srcpart a join srcpart b where a.ds = '2008-04-08' and a.hr = '11' and b.ds = '2008-04-08' and b.hr = '14' limit 5 PREHOOK: type: QUERY PREHOOK: Input: default@srcpart diff --git ql/src/test/results/clientpositive/input26.q.out ql/src/test/results/clientpositive/input26.q.out index 87b7081..2ac71e7 100644 --- ql/src/test/results/clientpositive/input26.q.out +++ ql/src/test/results/clientpositive/input26.q.out @@ -26,33 +26,29 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string), value (type: string), '2008-04-08' (type: string), '11' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - value expressions: _col1 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string) Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string) - outputColumnNames: _col0, _col1 + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 5 Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '11' (type: string) - outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 5 Data size: 50 Basic stats: COMPLETE 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 @@ -60,31 +56,23 @@ STAGE PLANS: TableScan Union Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 6 Data size: 50 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 + 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 TableScan Union Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '2008-04-08' (type: string), _col3 (type: string) - outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false Statistics: Num rows: 6 Data size: 50 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 6 Data size: 50 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 + 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-3 Map Reduce @@ -96,8 +84,8 @@ STAGE PLANS: predicate: ((ds = '2008-04-08') and (hr = '14')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string), value (type: string), '2008-04-08' (type: string), '14' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Limit Number of rows: 5 @@ -106,25 +94,21 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE TopN Hash Memory Usage: 0.1 - value expressions: _col0 (type: string), _col1 (type: string) + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) - outputColumnNames: _col0, _col1 + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Limit Number of rows: 5 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '14' (type: string) - outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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-0 Fetch Operator diff --git ql/src/test/results/clientpositive/input6.q.out ql/src/test/results/clientpositive/input6.q.out index 5ed2767..8ef9ab4 100644 --- ql/src/test/results/clientpositive/input6.q.out +++ ql/src/test/results/clientpositive/input6.q.out @@ -35,7 +35,7 @@ STAGE PLANS: predicate: key is null (type: boolean) Statistics: Num rows: 12 Data size: 91 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: null (type: string), value (type: string) + expressions: null (type: void), value (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 12 Data size: 91 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -109,7 +109,7 @@ INSERT OVERWRITE TABLE dest1 SELECT src1.key, src1.value WHERE src1.key is null POSTHOOK: type: QUERY POSTHOOK: Input: default@src1 POSTHOOK: Output: default@dest1 -POSTHOOK: Lineage: dest1.key SIMPLE [(src1)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.key SIMPLE [] POSTHOOK: Lineage: dest1.value SIMPLE [(src1)src1.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: SELECT dest1.* FROM dest1 PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/input9.q.out ql/src/test/results/clientpositive/input9.q.out index 38216a6..b00ed12 100644 --- ql/src/test/results/clientpositive/input9.q.out +++ ql/src/test/results/clientpositive/input9.q.out @@ -32,15 +32,15 @@ STAGE PLANS: alias: src1 Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: false (type: boolean) - Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + predicate: null (type: void) + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: null (type: void), UDFToInteger(key) (type: int) outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/input_part10.q.out ql/src/test/results/clientpositive/input_part10.q.out index c8fb37e..8455bb3 100644 --- ql/src/test/results/clientpositive/input_part10.q.out +++ ql/src/test/results/clientpositive/input_part10.q.out @@ -45,6 +45,8 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: int), 2 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 4000 Basic stats: COMPLETE Column stats: COMPLETE Limit Number of rows: 1 @@ -53,9 +55,10 @@ STAGE PLANS: sort order: Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: int), _col1 (type: int) Reduce Operator Tree: Select Operator - expressions: 1 (type: int), 2 (type: int) + expressions: VALUE._col0 (type: int), VALUE._col1 (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Limit diff --git ql/src/test/results/clientpositive/insert_into5.q.out ql/src/test/results/clientpositive/insert_into5.q.out index b9510b9..7b471f4 100644 --- ql/src/test/results/clientpositive/insert_into5.q.out +++ ql/src/test/results/clientpositive/insert_into5.q.out @@ -39,6 +39,8 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: int), 'one' (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 45500 Basic stats: COMPLETE Column stats: COMPLETE Limit Number of rows: 10 @@ -47,9 +49,10 @@ STAGE PLANS: sort order: Statistics: Num rows: 10 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: Select Operator - expressions: 1 (type: int), 'one' (type: string) + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 10 Data size: 910 Basic stats: COMPLETE Column stats: COMPLETE Limit diff --git ql/src/test/results/clientpositive/join28.q.out ql/src/test/results/clientpositive/join28.q.out index 9aadd28..82fb3e3 100644 --- ql/src/test/results/clientpositive/join28.q.out +++ ql/src/test/results/clientpositive/join28.q.out @@ -52,7 +52,7 @@ STAGE PLANS: alias: z Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: key is not null (type: boolean) + predicate: ((11.0 = 11.0) and key is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) diff --git ql/src/test/results/clientpositive/join32.q.out ql/src/test/results/clientpositive/join32.q.out index aad5463..dae4541 100644 --- ql/src/test/results/clientpositive/join32.q.out +++ ql/src/test/results/clientpositive/join32.q.out @@ -89,7 +89,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) diff --git ql/src/test/results/clientpositive/join32_lessSize.q.out ql/src/test/results/clientpositive/join32_lessSize.q.out index a94b951..01a3439 100644 --- ql/src/test/results/clientpositive/join32_lessSize.q.out +++ ql/src/test/results/clientpositive/join32_lessSize.q.out @@ -78,7 +78,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -1305,7 +1305,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -1821,7 +1821,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -2205,7 +2205,7 @@ STAGE PLANS: alias: x Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -2462,7 +2462,7 @@ STAGE PLANS: alias: y Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) diff --git ql/src/test/results/clientpositive/join33.q.out ql/src/test/results/clientpositive/join33.q.out index aad5463..dae4541 100644 --- ql/src/test/results/clientpositive/join33.q.out +++ ql/src/test/results/clientpositive/join33.q.out @@ -89,7 +89,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) diff --git ql/src/test/results/clientpositive/join8.q.out ql/src/test/results/clientpositive/join8.q.out index d7e7cb1..82b7357 100644 --- ql/src/test/results/clientpositive/join8.q.out +++ ql/src/test/results/clientpositive/join8.q.out @@ -153,7 +153,7 @@ POSTHOOK: Input: default@src POSTHOOK: Output: default@dest1 POSTHOOK: Lineage: dest1.c1 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: dest1.c2 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] -POSTHOOK: Lineage: dest1.c3 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.c3 EXPRESSION [] POSTHOOK: Lineage: dest1.c4 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: SELECT dest1.* FROM dest1 PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/join_filters_overlap.q.out ql/src/test/results/clientpositive/join_filters_overlap.q.out index ac00fd5..8a68d51 100644 --- ql/src/test/results/clientpositive/join_filters_overlap.q.out +++ ql/src/test/results/clientpositive/join_filters_overlap.q.out @@ -56,7 +56,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -77,7 +77,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -230,7 +230,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -268,7 +268,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -421,7 +421,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -459,7 +459,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -646,7 +646,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -667,7 +667,7 @@ STAGE PLANS: predicate: (value = 40) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 40 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -832,7 +832,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -853,7 +853,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -874,7 +874,7 @@ STAGE PLANS: predicate: (value = 40) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 40 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator diff --git ql/src/test/results/clientpositive/join_view.q.out ql/src/test/results/clientpositive/join_view.q.out index 57043fb..db68591 100644 --- ql/src/test/results/clientpositive/join_view.q.out +++ ql/src/test/results/clientpositive/join_view.q.out @@ -38,6 +38,7 @@ POSTHOOK: Input: default@invites POSTHOOK: Input: default@invites2 POSTHOOK: Output: database:default POSTHOOK: Output: default@v +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain select * from v where ds='2011-09-01' PREHOOK: type: QUERY POSTHOOK: query: explain select * from v where ds='2011-09-01' @@ -61,9 +62,7 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: '2011-09-01' (type: string) - sort order: + - Map-reduce partition columns: '2011-09-01' (type: string) + sort order: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string) TableScan @@ -77,9 +76,7 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: '2011-09-01' (type: string) - sort order: + - Map-reduce partition columns: '2011-09-01' (type: string) + sort order: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: int) Reduce Operator Tree: @@ -87,8 +84,8 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: string) - 1 _col1 (type: string) + 0 + 1 outputColumnNames: _col0, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator diff --git ql/src/test/results/clientpositive/lineage3.q.out ql/src/test/results/clientpositive/lineage3.q.out index 61acf52..14932b5 100644 --- ql/src/test/results/clientpositive/lineage3.q.out +++ ql/src/test/results/clientpositive/lineage3.q.out @@ -135,7 +135,7 @@ and x.ctinyint + length(c.cstring2) < 1000 PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc #### A masked pattern was here #### -{"version":"1.0","engine":"mr","database":"default","hash":"3a12ad24b2622a8958df12d0bdc60f8a","queryText":"select x.ctinyint, x.cint, c.cbigint-100, c.cstring1\nfrom alltypesorc c\njoin (\n select a.ctinyint ctinyint, b.cint cint\n from (select * from alltypesorc a where cboolean1=false) a\n join alltypesorc b on (a.cint = b.cbigint - 224870380)\n ) x on (x.cint = c.cint)\nwhere x.ctinyint > 10\nand x.cint < 4.5\nand x.ctinyint + length(c.cstring2) < 1000","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"expression":"(c.cbigint - UDFToLong(100))","edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3],"expression":"(UDFToDouble(c.cint) < 4.5)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(c.cint = c.cint)","edgeType":"PREDICATE"},{"sources":[5,6],"targets":[0,1,2,3],"expression":"((UDFToDouble(c.cint) < 4.5) and c.cbigint is not null)","edgeType":"PREDICATE"},{"sources":[6,5],"targets":[0,1,2,3],"expression":"((c.cbigint - UDFToLong(224870380)) = UDFToLong(c.cint))","edgeType":"PREDICATE"},{"sources":[8,4,5],"targets":[0,1,2,3],"expression":"((c.cboolean1 = false) and (c.ctinyint > 10) and c.cint is not null)","edgeType":"PREDICATE"},{"sources":[4,9],"targets":[0,1,2,3],"expression":"((UDFToInteger(c.ctinyint) + length(c.cstring2)) < 1000)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"x.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"x.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"c.cstring1"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"}]} +{"version":"1.0","engine":"mr","database":"default","hash":"3a12ad24b2622a8958df12d0bdc60f8a","queryText":"select x.ctinyint, x.cint, c.cbigint-100, c.cstring1\nfrom alltypesorc c\njoin (\n select a.ctinyint ctinyint, b.cint cint\n from (select * from alltypesorc a where cboolean1=false) a\n join alltypesorc b on (a.cint = b.cbigint - 224870380)\n ) x on (x.cint = c.cint)\nwhere x.ctinyint > 10\nand x.cint < 4.5\nand x.ctinyint + length(c.cstring2) < 1000","edges":[{"sources":[4],"targets":[0],"edgeType":"PROJECTION"},{"sources":[5],"targets":[1],"edgeType":"PROJECTION"},{"sources":[6],"targets":[2],"expression":"(c.cbigint - 100)","edgeType":"PROJECTION"},{"sources":[7],"targets":[3],"edgeType":"PROJECTION"},{"sources":[5],"targets":[0,1,2,3],"expression":"(UDFToDouble(c.cint) < 4.5)","edgeType":"PREDICATE"},{"sources":[5],"targets":[0,1,2,3],"expression":"(c.cint = c.cint)","edgeType":"PREDICATE"},{"sources":[5,6],"targets":[0,1,2,3],"expression":"((UDFToDouble(c.cint) < 4.5) and c.cbigint is not null)","edgeType":"PREDICATE"},{"sources":[6,5],"targets":[0,1,2,3],"expression":"((c.cbigint - 224870380) = UDFToLong(c.cint))","edgeType":"PREDICATE"},{"sources":[8,4,5],"targets":[0,1,2,3],"expression":"((c.cboolean1 = false) and (c.ctinyint > 10) and c.cint is not null)","edgeType":"PREDICATE"},{"sources":[4,9],"targets":[0,1,2,3],"expression":"((UDFToInteger(c.ctinyint) + length(c.cstring2)) < 1000)","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"x.ctinyint"},{"id":1,"vertexType":"COLUMN","vertexId":"x.cint"},{"id":2,"vertexType":"COLUMN","vertexId":"c2"},{"id":3,"vertexType":"COLUMN","vertexId":"c.cstring1"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"},{"id":5,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cint"},{"id":6,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cbigint"},{"id":7,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring1"},{"id":8,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cboolean1"},{"id":9,"vertexType":"COLUMN","vertexId":"default.alltypesorc.cstring2"}]} 11 -654374827 857266369 OEfPnHnIYueoup PREHOOK: query: select c1, x2, x3 from ( @@ -166,7 +166,7 @@ where key in (select key+18 from src1) order by key PREHOOK: type: QUERY PREHOOK: Input: default@src1 #### A masked pattern was here #### -{"version":"1.0","engine":"mr","database":"default","hash":"8b9d63653e36ecf4dd425d3cc3de9199","queryText":"select key, value from src1\nwhere key in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"UDFToDouble(src1.key) is not null","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + UDFToDouble(18)))","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) + UDFToDouble(18)) is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]} +{"version":"1.0","engine":"mr","database":"default","hash":"8b9d63653e36ecf4dd425d3cc3de9199","queryText":"select key, value from src1\nwhere key in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"UDFToDouble(src1.key) is not null","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + 18.0))","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) + 18.0) is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"}]} 146 val_146 273 val_273 PREHOOK: query: select * from src1 a @@ -178,15 +178,15 @@ PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Input: default@src1 #### A masked pattern was here #### -{"version":"1.0","engine":"mr","database":"default","hash":"8bf193b0658183be94e2428a79d91d10","queryText":"select * from src1 a\nwhere exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"((UDFToDouble(a.key) > UDFToDouble(300)) and UDFToDouble(a.key) is not null)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]} +{"version":"1.0","engine":"mr","database":"default","hash":"8bf193b0658183be94e2428a79d91d10","queryText":"select * from src1 a\nwhere exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"((UDFToDouble(a.key) > 300.0) and UDFToDouble(a.key) is not null)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"UDFToDouble((UDFToInteger(b.ctinyint) + 300)) is not null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]} 311 val_311 -Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[15][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select key, value from src1 where key not in (select key+18 from src1) order by key PREHOOK: type: QUERY PREHOOK: Input: default@src1 #### A masked pattern was here #### -{"version":"1.0","engine":"mr","database":"default","hash":"9b488fe1d7cf018aad3825173808cd36","queryText":"select key, value from src1\nwhere key not in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) + UDFToDouble(18)) is null","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(count(*) = 0)","edgeType":"PREDICATE"},{"sources":[],"targets":[0,1],"expression":"true","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + UDFToDouble(18)))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"TABLE","vertexId":"default.src1"}]} +{"version":"1.0","engine":"mr","database":"default","hash":"9b488fe1d7cf018aad3825173808cd36","queryText":"select key, value from src1\nwhere key not in (select key+18 from src1) order by key","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) + 18.0) is null","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(count(*) = 0)","edgeType":"PREDICATE"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(src1.key) = (UDFToDouble(src1.key) + 18.0))","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"key"},{"id":1,"vertexType":"COLUMN","vertexId":"value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"TABLE","vertexId":"default.src1"}]} PREHOOK: query: select * from src1 a where not exists (select cint from alltypesorc b @@ -196,7 +196,7 @@ PREHOOK: type: QUERY PREHOOK: Input: default@alltypesorc PREHOOK: Input: default@src1 #### A masked pattern was here #### -{"version":"1.0","engine":"mr","database":"default","hash":"53191056e05af9080a30de853e8cea9c","queryText":"select * from src1 a\nwhere not exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > UDFToDouble(300))","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(UDFToInteger(b.ctinyint) + 300) is null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]} +{"version":"1.0","engine":"mr","database":"default","hash":"53191056e05af9080a30de853e8cea9c","queryText":"select * from src1 a\nwhere not exists\n (select cint from alltypesorc b\n where a.key = b.ctinyint + 300)\nand key > 300","edges":[{"sources":[2],"targets":[0],"edgeType":"PROJECTION"},{"sources":[3],"targets":[1],"edgeType":"PROJECTION"},{"sources":[2],"targets":[0,1],"expression":"(UDFToDouble(a.key) > 300.0)","edgeType":"PREDICATE"},{"sources":[2,4],"targets":[0,1],"expression":"(UDFToDouble(a.key) = UDFToDouble((UDFToInteger(b.ctinyint) + 300)))","edgeType":"PREDICATE"},{"sources":[4],"targets":[0,1],"expression":"(UDFToInteger(b.ctinyint) + 300) is null","edgeType":"PREDICATE"}],"vertices":[{"id":0,"vertexType":"COLUMN","vertexId":"a.key"},{"id":1,"vertexType":"COLUMN","vertexId":"a.value"},{"id":2,"vertexType":"COLUMN","vertexId":"default.src1.key"},{"id":3,"vertexType":"COLUMN","vertexId":"default.src1.value"},{"id":4,"vertexType":"COLUMN","vertexId":"default.alltypesorc.ctinyint"}]} 369 401 val_401 406 val_406 diff --git ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out index 8517a52..b994099 100644 --- ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out +++ ql/src/test/results/clientpositive/list_bucket_query_oneskew_2.q.out @@ -76,7 +76,7 @@ SELECT key+11, value FROM src WHERE key=484 POSTHOOK: type: QUERY POSTHOOK: Input: default@src POSTHOOK: Output: default@fact_tz@ds=1/hr=2 -POSTHOOK: Lineage: fact_tz PARTITION(ds=1,hr=2).x EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: fact_tz PARTITION(ds=1,hr=2).x EXPRESSION [] POSTHOOK: Lineage: fact_tz PARTITION(ds=1,hr=2).y SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ] #### A masked pattern was here #### PREHOOK: query: -- switch fact_daily to skewed table and point its location to /fact_tz/ds=1 @@ -517,10 +517,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 484 (type: int) + key expressions: _col0 (type: int) null sort order: a sort order: + - Map-reduce partition columns: 484 (type: int) + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: -1 value expressions: _col1 (type: bigint) @@ -577,36 +577,32 @@ STAGE PLANS: Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: 484 (type: int) + keys: KEY._col0 (type: int) mode: mergepartial outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 484 (type: int), _col1 (type: bigint) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 - Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE -#### A masked pattern was here #### - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types int:bigint - escape.delim \ - hive.serialization.extend.additional.nesting.levels true - serialization.escape.crlf true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1 + columns.types int:bigint + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.escape.crlf true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out index 0c6fb63..a802e30 100644 --- ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out +++ ql/src/test/results/clientpositive/list_bucket_query_oneskew_3.q.out @@ -259,12 +259,12 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: (x <> 86) (type: boolean) - Statistics: Num rows: 29 Data size: 117 Basic stats: COMPLETE Column stats: NONE + predicate: (not (x = 86)) (type: boolean) + Statistics: Num rows: 15 Data size: 60 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: x (type: int) outputColumnNames: _col0 - Statistics: Num rows: 29 Data size: 117 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 15 Data size: 60 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: -- List Bucketing Query diff --git ql/src/test/results/clientpositive/llap/cte_mat_1.q.out ql/src/test/results/clientpositive/llap/cte_mat_1.q.out index 9ba574d..063b026 100644 --- ql/src/test/results/clientpositive/llap/cte_mat_1.q.out +++ ql/src/test/results/clientpositive/llap/cte_mat_1.q.out @@ -65,18 +65,15 @@ STAGE PLANS: keys: 0 '5' (type: string) 1 '5' (type: string) + outputColumnNames: _col0 Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '5' (type: string) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 275 Data size: 2921 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 + 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 diff --git ql/src/test/results/clientpositive/llap/cte_mat_2.q.out ql/src/test/results/clientpositive/llap/cte_mat_2.q.out index 9ba574d..063b026 100644 --- ql/src/test/results/clientpositive/llap/cte_mat_2.q.out +++ ql/src/test/results/clientpositive/llap/cte_mat_2.q.out @@ -65,18 +65,15 @@ STAGE PLANS: keys: 0 '5' (type: string) 1 '5' (type: string) + outputColumnNames: _col0 Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '5' (type: string) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 275 Data size: 2921 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 + 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 diff --git ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out index fe2049f..b5b1c61 100644 --- ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out +++ ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out @@ -1633,12 +1633,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1662,7 +1662,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1742,9 +1742,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -1790,7 +1790,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1891,9 +1891,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -1905,7 +1905,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1985,9 +1985,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -2018,7 +2018,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -2112,9 +2112,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + key expressions: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) sort order: + - Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -2160,7 +2160,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + 0 UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) 1 UDFToString(_col0) (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -2275,9 +2275,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -2289,7 +2289,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -2319,32 +2319,30 @@ STAGE PLANS: Execution mode: llap Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -2382,7 +2380,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 #### A masked pattern was here #### 1000 -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: -- non-equi join EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY @@ -2483,7 +2481,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY PREHOOK: Input: default@srcpart @@ -3006,17 +3004,21 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Execution mode: llap LLAP IO: no inputs Map 5 @@ -4748,7 +4750,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) input vertices: 1 Map 3 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -4778,12 +4780,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -4871,7 +4873,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) input vertices: 1 Map 3 @@ -5012,7 +5014,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) input vertices: 1 Reducer 4 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -5040,9 +5042,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -5065,32 +5067,30 @@ STAGE PLANS: Execution mode: llap Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -5440,41 +5440,45 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col1 - input vertices: - 1 Map 3 - Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE - HybridGraceHashJoin: true + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: string) + 0 _col0 (type: string) 1 _col0 (type: string) + outputColumnNames: _col1 input vertices: - 1 Map 4 - Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE HybridGraceHashJoin: true - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + HybridGraceHashJoin: true + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) Execution mode: llap LLAP IO: no inputs Map 3 diff --git ql/src/test/results/clientpositive/llap/tez_self_join.q.out ql/src/test/results/clientpositive/llap/tez_self_join.q.out index 68f231c..bd9c96c 100644 --- ql/src/test/results/clientpositive/llap/tez_self_join.q.out +++ ql/src/test/results/clientpositive/llap/tez_self_join.q.out @@ -42,6 +42,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@values__tmp__table__2 POSTHOOK: Output: default@tez_self_join2 POSTHOOK: Lineage: tez_self_join2.id1 EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: explain select s.id2, s.id3 from @@ -90,9 +91,7 @@ STAGE PLANS: outputColumnNames: _col0, _col2 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'ab' (type: string) - sort order: + - Map-reduce partition columns: 'ab' (type: string) + sort order: Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col2 (type: string) Execution mode: llap @@ -108,9 +107,7 @@ STAGE PLANS: Select Operator Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'ab' (type: string) - sort order: + - Map-reduce partition columns: 'ab' (type: string) + sort order: Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -140,16 +137,20 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 'ab' (type: string) - 1 'ab' (type: string) + 0 + 1 outputColumnNames: _col0, _col2 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: int) - sort order: + - Map-reduce partition columns: _col0 (type: int) + Select Operator + expressions: _col0 (type: int), 'ab' (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: string) + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: string) Reducer 3 Execution mode: llap Reduce Operator Tree: @@ -159,10 +160,10 @@ STAGE PLANS: keys: 0 _col0 (type: int) 1 _col0 (type: int) - outputColumnNames: _col2 + outputColumnNames: _col1, _col2 Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: 'ab' (type: string), _col2 (type: string) + expressions: _col1 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -179,6 +180,7 @@ STAGE PLANS: Processor Tree: ListSink +Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select s.id2, s.id3 from ( diff --git ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out index 14a273b..ce86640 100644 --- ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out +++ ql/src/test/results/clientpositive/llap/tez_union_dynamic_partition.q.out @@ -66,21 +66,17 @@ STAGE PLANS: alias: dummy Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: 1 (type: int) - outputColumnNames: _col0 + expressions: 1 (type: int), '2014' (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), '2014' (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - 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 - name: default.partunion1 + 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 + name: default.partunion1 Execution mode: llap LLAP IO: no inputs Map 3 @@ -89,21 +85,17 @@ STAGE PLANS: alias: dummy Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: 2 (type: int) - outputColumnNames: _col0 + expressions: 2 (type: int), '2014' (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), '2014' (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - 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 - name: default.partunion1 + 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 + name: default.partunion1 Execution mode: llap LLAP IO: no inputs Union 2 diff --git ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out index 5f52822..7cde14e 100644 --- ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out +++ ql/src/test/results/clientpositive/llap/vectorized_dynamic_partition_pruning.q.out @@ -1388,12 +1388,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1417,7 +1417,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1497,9 +1497,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -1545,7 +1545,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1646,9 +1646,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized, llap LLAP IO: all inputs @@ -1660,7 +1660,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1740,9 +1740,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -1773,7 +1773,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1867,9 +1867,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + key expressions: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) sort order: + - Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -1915,7 +1915,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + 0 UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) 1 UDFToString(_col0) (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -2030,9 +2030,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -2044,7 +2044,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -2074,32 +2074,30 @@ STAGE PLANS: Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -2137,7 +2135,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 #### A masked pattern was here #### 1000 -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: -- non-equi join EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY @@ -2238,7 +2236,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY PREHOOK: Input: default@srcpart @@ -2761,17 +2759,21 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Execution mode: llap LLAP IO: no inputs Map 5 @@ -4377,7 +4379,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) input vertices: 1 Map 3 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -4407,12 +4409,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -4500,7 +4502,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) input vertices: 1 Map 3 @@ -4641,7 +4643,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) input vertices: 1 Reducer 4 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -4669,9 +4671,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Execution mode: llap LLAP IO: no inputs @@ -4694,32 +4696,30 @@ STAGE PLANS: Execution mode: vectorized, llap Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -5069,41 +5069,45 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col1 - input vertices: - 1 Map 3 - Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE - HybridGraceHashJoin: true + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: string) + 0 _col0 (type: string) 1 _col0 (type: string) + outputColumnNames: _col1 input vertices: - 1 Map 4 - Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE HybridGraceHashJoin: true - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + HybridGraceHashJoin: true + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) Execution mode: llap LLAP IO: no inputs Map 3 diff --git ql/src/test/results/clientpositive/load_dyn_part14.q.out ql/src/test/results/clientpositive/load_dyn_part14.q.out index 53e9df3..772e82e 100644 --- ql/src/test/results/clientpositive/load_dyn_part14.q.out +++ ql/src/test/results/clientpositive/load_dyn_part14.q.out @@ -74,6 +74,8 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: null (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Limit Number of rows: 2 @@ -82,20 +84,25 @@ STAGE PLANS: sort order: Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string) Reduce Operator Tree: - Limit - Number of rows: 2 + Select Operator + expressions: VALUE._col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - expressions: 'k1' (type: string), null (type: string) - outputColumnNames: _col0, _col1 + Limit + Number of rows: 2 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - 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 + Select Operator + expressions: 'k1' (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + 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 diff --git ql/src/test/results/clientpositive/mapjoin_subquery.q.out ql/src/test/results/clientpositive/mapjoin_subquery.q.out index 9640179..1ae7026 100644 --- ql/src/test/results/clientpositive/mapjoin_subquery.q.out +++ ql/src/test/results/clientpositive/mapjoin_subquery.q.out @@ -39,7 +39,7 @@ STAGE PLANS: alias: z Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: key is not null (type: boolean) + predicate: ((11.0 = 11.0) and key is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) @@ -282,7 +282,7 @@ STAGE PLANS: alias: z Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: key is not null (type: boolean) + predicate: ((11.0 = 11.0) and key is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) diff --git ql/src/test/results/clientpositive/partition_multilevels.q.out ql/src/test/results/clientpositive/partition_multilevels.q.out index 948d3a0..96aa76f 100644 --- ql/src/test/results/clientpositive/partition_multilevels.q.out +++ ql/src/test/results/clientpositive/partition_multilevels.q.out @@ -991,29 +991,29 @@ STAGE PLANS: Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: level2 (type: string), level3 (type: string) - outputColumnNames: _col1, _col2 + outputColumnNames: level2, level3 Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() - keys: '2222' (type: string), _col1 (type: string), _col2 (type: string) + keys: level2 (type: string), level3 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2222' (type: string), _col1 (type: string), _col2 (type: string) - sort order: +++ - Map-reduce partition columns: '2222' (type: string), _col1 (type: string), _col2 (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: '2222' (type: string), KEY._col1 (type: string), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 54 Data size: 573 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: '2222' (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: bigint) + expressions: '2222' (type: string), _col0 (type: string), _col1 (type: string), _col2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 54 Data size: 573 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1583,29 +1583,29 @@ STAGE PLANS: Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: level2 (type: string), level3 (type: string) - outputColumnNames: _col1, _col2 + outputColumnNames: level2, level3 Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() - keys: '2222' (type: string), _col1 (type: string), _col2 (type: string) + keys: level2 (type: string), level3 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2222' (type: string), _col1 (type: string), _col2 (type: string) - sort order: +++ - Map-reduce partition columns: '2222' (type: string), _col1 (type: string), _col2 (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 108 Data size: 1146 Basic stats: COMPLETE Column stats: NONE - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: '2222' (type: string), KEY._col1 (type: string), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 54 Data size: 573 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: '2222' (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: bigint) + expressions: '2222' (type: string), _col0 (type: string), _col1 (type: string), _col2 (type: bigint) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 54 Data size: 573 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/pcr.q.out ql/src/test/results/clientpositive/pcr.q.out index b53226e..afa39f2 100644 --- ql/src/test/results/clientpositive/pcr.q.out +++ ql/src/test/results/clientpositive/pcr.q.out @@ -1505,7 +1505,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: 14 (type: int), KEY.reducesinkkey1 (type: string) + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -2327,23 +2327,27 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: int), _col4 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string - escape.delim \ - serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4 + columns.types int,string,string,int,string + escape.delim \ + serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-2 Map Reduce @@ -2356,7 +2360,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col1 (type: string), _col3 (type: int), _col4 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2367,8 +2371,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4 + columns.types int,string,string,int,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -2376,8 +2380,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4 + columns.types int,string,string,int,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -2386,7 +2390,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), '2000-04-08' (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col1 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -2627,23 +2631,27 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: int), _col4 (type: string), '2000-04-09' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string - escape.delim \ - serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4,_col5 + columns.types int,string,string,int,string,string + escape.delim \ + serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-2 Map Reduce @@ -2656,7 +2664,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col1 (type: string), _col3 (type: int), _col4 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string), _col5 (type: string) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -2667,8 +2675,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4,_col5 + columns.types int,string,string,int,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -2676,8 +2684,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4,_col5 + columns.types int,string,string,int,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -2686,7 +2694,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), '2000-04-09' (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4765,7 +4773,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey2 (type: string) + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4944,7 +4952,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), '11' (type: string) + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/pcs.q.out ql/src/test/results/clientpositive/pcs.q.out index 8b99401..67a3718 100644 --- ql/src/test/results/clientpositive/pcs.q.out +++ ql/src/test/results/clientpositive/pcs.q.out @@ -1208,7 +1208,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: (const struct(10)) IN (const struct(10), const struct(11)) (type: boolean) + predicate: (struct(CASE WHEN (true) THEN (10) ELSE (20) END)) IN (const struct(10), const struct(11)) (type: boolean) Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: ds (type: string) diff --git ql/src/test/results/clientpositive/perf/query31.q.out ql/src/test/results/clientpositive/perf/query31.q.out index 3cb7079..2a6eb3d 100644 --- ql/src/test/results/clientpositive/perf/query31.q.out +++ ql/src/test/results/clientpositive/perf/query31.q.out @@ -32,302 +32,292 @@ Stage-0 limit:-1 Stage-1 Reducer 7 - File Output Operator [FS_141] - Select Operator [SEL_140] (rows=26620001 width=1014) + File Output Operator [FS_135] + Select Operator [SEL_134] (rows=26620001 width=1014) Output:["_col0","_col1","_col2","_col3","_col4","_col5"] <-Reducer 6 [SIMPLE_EDGE] - SHUFFLE [RS_139] - Select Operator [SEL_138] (rows=26620001 width=1014) - Output:["_col0","_col2","_col3","_col4","_col5"] - Filter Operator [FIL_137] (rows=26620001 width=1014) - predicate:CASE WHEN ((_col7 > 0)) THEN (CASE WHEN ((_col19 > 0)) THEN (((_col23 / _col19) > (_col11 / _col7))) ELSE ((null > (_col11 / _col7))) END) ELSE (CASE WHEN ((_col19 > 0)) THEN (((_col23 / _col19) > null)) ELSE (null) END) END - Merge Join Operator [MERGEJOIN_281] (rows=53240002 width=1014) - Conds:RS_134._col12=RS_135._col0(Inner),Output:["_col0","_col3","_col7","_col11","_col15","_col19","_col23"] + SHUFFLE [RS_133] + Select Operator [SEL_132] (rows=26620001 width=1014) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] + Filter Operator [FIL_131] (rows=26620001 width=1014) + predicate:CASE WHEN ((_col4 > 0)) THEN (CASE WHEN ((_col10 > 0)) THEN (((_col12 / _col10) > (_col6 / _col4))) ELSE ((null > (_col6 / _col4))) END) ELSE (CASE WHEN ((_col10 > 0)) THEN (((_col12 / _col10) > null)) ELSE (null) END) END + Merge Join Operator [MERGEJOIN_275] (rows=53240002 width=1014) + Conds:RS_128._col7=RS_129._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col6","_col8","_col10","_col12"] <-Reducer 37 [SIMPLE_EDGE] - SHUFFLE [RS_135] + SHUFFLE [RS_129] PartitionCols:_col0 - Select Operator [SEL_133] (rows=22000000 width=1014) - Output:["_col0","_col3"] - Group By Operator [GBY_132] (rows=22000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, 3, 1998 - <-Reducer 36 [SIMPLE_EDGE] - SHUFFLE [RS_131] - PartitionCols:_col0, 3, 1998 - Group By Operator [GBY_130] (rows=44000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:_col0, 3, 1998 - Select Operator [SEL_128] (rows=44000000 width=1014) - Output:["_col0","_col3"] - Merge Join Operator [MERGEJOIN_279] (rows=44000000 width=1014) - Conds:RS_125._col1=RS_126._col0(Inner),Output:["_col2","_col7"] - <-Map 39 [SIMPLE_EDGE] - SHUFFLE [RS_126] - PartitionCols:_col0 - Select Operator [SEL_121] (rows=40000000 width=1014) - Output:["_col0","_col1"] - Filter Operator [FIL_267] (rows=40000000 width=1014) - predicate:(ca_address_sk is not null and ca_county is not null) - TableScan [TS_119] (rows=40000000 width=1014) - default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] - <-Reducer 35 [SIMPLE_EDGE] - SHUFFLE [RS_125] - PartitionCols:_col1 - Merge Join Operator [MERGEJOIN_278] (rows=20088 width=1119) - Conds:RS_122._col0=RS_123._col0(Inner),Output:["_col1","_col2"] - <-Map 34 [SIMPLE_EDGE] - SHUFFLE [RS_122] - PartitionCols:_col0 - Select Operator [SEL_115] (rows=1 width=0) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_265] (rows=1 width=0) - predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null) - TableScan [TS_113] (rows=1 width=0) - default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"] - <-Map 38 [SIMPLE_EDGE] - SHUFFLE [RS_123] - PartitionCols:_col0 - Select Operator [SEL_118] (rows=18262 width=1119) - Output:["_col0"] - Filter Operator [FIL_266] (rows=18262 width=1119) - predicate:((d_qoy = 3) and (d_year = 1998) and d_date_sk is not null) - TableScan [TS_116] (rows=73049 width=1119) - default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] + Group By Operator [GBY_126] (rows=22000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0 + <-Reducer 36 [SIMPLE_EDGE] + SHUFFLE [RS_125] + PartitionCols:_col0 + Group By Operator [GBY_124] (rows=44000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7 + Select Operator [SEL_123] (rows=44000000 width=1014) + Output:["_col7","_col2"] + Merge Join Operator [MERGEJOIN_273] (rows=44000000 width=1014) + Conds:RS_120._col1=RS_121._col0(Inner),Output:["_col2","_col7"] + <-Map 39 [SIMPLE_EDGE] + SHUFFLE [RS_121] + PartitionCols:_col0 + Select Operator [SEL_116] (rows=40000000 width=1014) + Output:["_col0","_col1"] + Filter Operator [FIL_261] (rows=40000000 width=1014) + predicate:(ca_address_sk is not null and ca_county is not null) + TableScan [TS_114] (rows=40000000 width=1014) + default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] + <-Reducer 35 [SIMPLE_EDGE] + SHUFFLE [RS_120] + PartitionCols:_col1 + Merge Join Operator [MERGEJOIN_272] (rows=20088 width=1119) + Conds:RS_117._col0=RS_118._col0(Inner),Output:["_col1","_col2"] + <-Map 34 [SIMPLE_EDGE] + SHUFFLE [RS_117] + PartitionCols:_col0 + Select Operator [SEL_110] (rows=1 width=0) + Output:["_col0","_col1","_col2"] + Filter Operator [FIL_259] (rows=1 width=0) + predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null) + TableScan [TS_108] (rows=1 width=0) + default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"] + <-Map 38 [SIMPLE_EDGE] + SHUFFLE [RS_118] + PartitionCols:_col0 + Select Operator [SEL_113] (rows=18262 width=1119) + Output:["_col0"] + Filter Operator [FIL_260] (rows=18262 width=1119) + predicate:((d_qoy = 3) and (d_year = 1998) and d_date_sk is not null) + TableScan [TS_111] (rows=73049 width=1119) + default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] <-Reducer 5 [SIMPLE_EDGE] - SHUFFLE [RS_134] - PartitionCols:_col12 - Select Operator [SEL_112] (rows=48400001 width=1014) - Output:["_col0","_col11","_col12","_col15","_col19","_col3","_col7"] - Filter Operator [FIL_111] (rows=48400001 width=1014) - predicate:CASE WHEN ((_col3 > 0)) THEN (CASE WHEN ((_col15 > 0)) THEN (((_col19 / _col15) > (_col7 / _col3))) ELSE ((null > (_col7 / _col3))) END) ELSE (CASE WHEN ((_col15 > 0)) THEN (((_col19 / _col15) > null)) ELSE (null) END) END - Merge Join Operator [MERGEJOIN_280] (rows=96800002 width=1014) - Conds:RS_105._col0=RS_106._col0(Inner),RS_106._col0=RS_107._col0(Inner),RS_105._col0=RS_108._col0(Inner),RS_108._col0=RS_109._col0(Inner),Output:["_col0","_col3","_col7","_col11","_col12","_col15","_col19"] + SHUFFLE [RS_128] + PartitionCols:_col7 + Select Operator [SEL_107] (rows=48400001 width=1014) + Output:["_col0","_col1","_col10","_col2","_col4","_col6","_col7","_col8"] + Filter Operator [FIL_106] (rows=48400001 width=1014) + predicate:CASE WHEN ((_col2 > 0)) THEN (CASE WHEN ((_col8 > 0)) THEN (((_col10 / _col8) > (_col4 / _col2))) ELSE ((null > (_col4 / _col2))) END) ELSE (CASE WHEN ((_col8 > 0)) THEN (((_col10 / _col8) > null)) ELSE (null) END) END + Merge Join Operator [MERGEJOIN_274] (rows=96800002 width=1014) + Conds:RS_100._col0=RS_101._col0(Inner),RS_101._col0=RS_102._col0(Inner),RS_100._col0=RS_103._col0(Inner),RS_103._col0=RS_104._col0(Inner),Output:["_col0","_col1","_col2","_col4","_col6","_col7","_col8","_col10"] <-Reducer 13 [SIMPLE_EDGE] - SHUFFLE [RS_106] + SHUFFLE [RS_101] PartitionCols:_col0 - Select Operator [SEL_41] (rows=22000000 width=1014) - Output:["_col0","_col3"] - Group By Operator [GBY_40] (rows=22000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, 2, 1998 - <-Reducer 12 [SIMPLE_EDGE] - SHUFFLE [RS_39] - PartitionCols:_col0, 2, 1998 - Group By Operator [GBY_38] (rows=44000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:_col0, 2, 1998 - Select Operator [SEL_36] (rows=44000000 width=1014) - Output:["_col0","_col3"] - Merge Join Operator [MERGEJOIN_271] (rows=44000000 width=1014) - Conds:RS_33._col1=RS_34._col0(Inner),Output:["_col2","_col7"] - <-Map 15 [SIMPLE_EDGE] - SHUFFLE [RS_34] - PartitionCols:_col0 - Select Operator [SEL_29] (rows=40000000 width=1014) - Output:["_col0","_col1"] - Filter Operator [FIL_255] (rows=40000000 width=1014) - predicate:(ca_address_sk is not null and ca_county is not null) - TableScan [TS_27] (rows=40000000 width=1014) - default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] - <-Reducer 11 [SIMPLE_EDGE] - SHUFFLE [RS_33] - PartitionCols:_col1 - Merge Join Operator [MERGEJOIN_270] (rows=20088 width=1119) - Conds:RS_30._col0=RS_31._col0(Inner),Output:["_col1","_col2"] - <-Map 10 [SIMPLE_EDGE] - SHUFFLE [RS_30] - PartitionCols:_col0 - Select Operator [SEL_23] (rows=1 width=0) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_253] (rows=1 width=0) - predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null) - TableScan [TS_21] (rows=1 width=0) - default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_addr_sk","ss_ext_sales_price"] - <-Map 14 [SIMPLE_EDGE] - SHUFFLE [RS_31] - PartitionCols:_col0 - Select Operator [SEL_26] (rows=18262 width=1119) - Output:["_col0"] - Filter Operator [FIL_254] (rows=18262 width=1119) - predicate:((d_qoy = 2) and (d_year = 1998) and d_date_sk is not null) - TableScan [TS_24] (rows=73049 width=1119) - default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] + Group By Operator [GBY_38] (rows=22000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0 + <-Reducer 12 [SIMPLE_EDGE] + SHUFFLE [RS_37] + PartitionCols:_col0 + Group By Operator [GBY_36] (rows=44000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7 + Select Operator [SEL_35] (rows=44000000 width=1014) + Output:["_col7","_col2"] + Merge Join Operator [MERGEJOIN_265] (rows=44000000 width=1014) + Conds:RS_32._col1=RS_33._col0(Inner),Output:["_col2","_col7"] + <-Map 15 [SIMPLE_EDGE] + SHUFFLE [RS_33] + PartitionCols:_col0 + Select Operator [SEL_28] (rows=40000000 width=1014) + Output:["_col0","_col1"] + Filter Operator [FIL_249] (rows=40000000 width=1014) + predicate:(ca_address_sk is not null and ca_county is not null) + TableScan [TS_26] (rows=40000000 width=1014) + default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] + <-Reducer 11 [SIMPLE_EDGE] + SHUFFLE [RS_32] + PartitionCols:_col1 + Merge Join Operator [MERGEJOIN_264] (rows=20088 width=1119) + Conds:RS_29._col0=RS_30._col0(Inner),Output:["_col1","_col2"] + <-Map 10 [SIMPLE_EDGE] + SHUFFLE [RS_29] + PartitionCols:_col0 + Select Operator [SEL_22] (rows=1 width=0) + Output:["_col0","_col1","_col2"] + Filter Operator [FIL_247] (rows=1 width=0) + predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null) + TableScan [TS_20] (rows=1 width=0) + default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_addr_sk","ss_ext_sales_price"] + <-Map 14 [SIMPLE_EDGE] + SHUFFLE [RS_30] + PartitionCols:_col0 + Select Operator [SEL_25] (rows=18262 width=1119) + Output:["_col0"] + Filter Operator [FIL_248] (rows=18262 width=1119) + predicate:((d_qoy = 2) and (d_year = 1998) and d_date_sk is not null) + TableScan [TS_23] (rows=73049 width=1119) + default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] <-Reducer 19 [SIMPLE_EDGE] - SHUFFLE [RS_107] + SHUFFLE [RS_102] PartitionCols:_col0 - Select Operator [SEL_62] (rows=22000000 width=1014) - Output:["_col0","_col3"] - Group By Operator [GBY_61] (rows=22000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, 3, 1998 - <-Reducer 18 [SIMPLE_EDGE] - SHUFFLE [RS_60] - PartitionCols:_col0, 3, 1998 - Group By Operator [GBY_59] (rows=44000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:_col0, 3, 1998 - Select Operator [SEL_57] (rows=44000000 width=1014) - Output:["_col0","_col3"] - Merge Join Operator [MERGEJOIN_273] (rows=44000000 width=1014) - Conds:RS_54._col1=RS_55._col0(Inner),Output:["_col2","_col7"] - <-Map 21 [SIMPLE_EDGE] - SHUFFLE [RS_55] - PartitionCols:_col0 - Select Operator [SEL_50] (rows=40000000 width=1014) - Output:["_col0","_col1"] - Filter Operator [FIL_258] (rows=40000000 width=1014) - predicate:(ca_address_sk is not null and ca_county is not null) - TableScan [TS_48] (rows=40000000 width=1014) - default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] - <-Reducer 17 [SIMPLE_EDGE] - SHUFFLE [RS_54] - PartitionCols:_col1 - Merge Join Operator [MERGEJOIN_272] (rows=20088 width=1119) - Conds:RS_51._col0=RS_52._col0(Inner),Output:["_col1","_col2"] - <-Map 16 [SIMPLE_EDGE] - SHUFFLE [RS_51] - PartitionCols:_col0 - Select Operator [SEL_44] (rows=1 width=0) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_256] (rows=1 width=0) - predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null) - TableScan [TS_42] (rows=1 width=0) - default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_addr_sk","ss_ext_sales_price"] - <-Map 20 [SIMPLE_EDGE] - SHUFFLE [RS_52] - PartitionCols:_col0 - Select Operator [SEL_47] (rows=18262 width=1119) - Output:["_col0"] - Filter Operator [FIL_257] (rows=18262 width=1119) - predicate:((d_qoy = 3) and (d_year = 1998) and d_date_sk is not null) - TableScan [TS_45] (rows=73049 width=1119) - default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] + Group By Operator [GBY_58] (rows=22000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0 + <-Reducer 18 [SIMPLE_EDGE] + SHUFFLE [RS_57] + PartitionCols:_col0 + Group By Operator [GBY_56] (rows=44000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7 + Select Operator [SEL_55] (rows=44000000 width=1014) + Output:["_col7","_col2"] + Merge Join Operator [MERGEJOIN_267] (rows=44000000 width=1014) + Conds:RS_52._col1=RS_53._col0(Inner),Output:["_col2","_col7"] + <-Map 21 [SIMPLE_EDGE] + SHUFFLE [RS_53] + PartitionCols:_col0 + Select Operator [SEL_48] (rows=40000000 width=1014) + Output:["_col0","_col1"] + Filter Operator [FIL_252] (rows=40000000 width=1014) + predicate:(ca_address_sk is not null and ca_county is not null) + TableScan [TS_46] (rows=40000000 width=1014) + default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] + <-Reducer 17 [SIMPLE_EDGE] + SHUFFLE [RS_52] + PartitionCols:_col1 + Merge Join Operator [MERGEJOIN_266] (rows=20088 width=1119) + Conds:RS_49._col0=RS_50._col0(Inner),Output:["_col1","_col2"] + <-Map 16 [SIMPLE_EDGE] + SHUFFLE [RS_49] + PartitionCols:_col0 + Select Operator [SEL_42] (rows=1 width=0) + Output:["_col0","_col1","_col2"] + Filter Operator [FIL_250] (rows=1 width=0) + predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null) + TableScan [TS_40] (rows=1 width=0) + default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_addr_sk","ss_ext_sales_price"] + <-Map 20 [SIMPLE_EDGE] + SHUFFLE [RS_50] + PartitionCols:_col0 + Select Operator [SEL_45] (rows=18262 width=1119) + Output:["_col0"] + Filter Operator [FIL_251] (rows=18262 width=1119) + predicate:((d_qoy = 3) and (d_year = 1998) and d_date_sk is not null) + TableScan [TS_43] (rows=73049 width=1119) + default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] <-Reducer 25 [SIMPLE_EDGE] - SHUFFLE [RS_108] + SHUFFLE [RS_103] PartitionCols:_col0 - Select Operator [SEL_83] (rows=22000000 width=1014) - Output:["_col0","_col3"] - Group By Operator [GBY_82] (rows=22000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, 1, 1998 - <-Reducer 24 [SIMPLE_EDGE] - SHUFFLE [RS_81] - PartitionCols:_col0, 1, 1998 - Group By Operator [GBY_80] (rows=44000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:_col0, 1, 1998 - Select Operator [SEL_78] (rows=44000000 width=1014) - Output:["_col0","_col3"] - Merge Join Operator [MERGEJOIN_275] (rows=44000000 width=1014) - Conds:RS_75._col1=RS_76._col0(Inner),Output:["_col2","_col7"] - <-Map 27 [SIMPLE_EDGE] - SHUFFLE [RS_76] - PartitionCols:_col0 - Select Operator [SEL_71] (rows=40000000 width=1014) - Output:["_col0","_col1"] - Filter Operator [FIL_261] (rows=40000000 width=1014) - predicate:(ca_address_sk is not null and ca_county is not null) - TableScan [TS_69] (rows=40000000 width=1014) - default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] - <-Reducer 23 [SIMPLE_EDGE] - SHUFFLE [RS_75] - PartitionCols:_col1 - Merge Join Operator [MERGEJOIN_274] (rows=20088 width=1119) - Conds:RS_72._col0=RS_73._col0(Inner),Output:["_col1","_col2"] - <-Map 22 [SIMPLE_EDGE] - SHUFFLE [RS_72] - PartitionCols:_col0 - Select Operator [SEL_65] (rows=1 width=0) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_259] (rows=1 width=0) - predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null) - TableScan [TS_63] (rows=1 width=0) - default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"] - <-Map 26 [SIMPLE_EDGE] - SHUFFLE [RS_73] - PartitionCols:_col0 - Select Operator [SEL_68] (rows=18262 width=1119) - Output:["_col0"] - Filter Operator [FIL_260] (rows=18262 width=1119) - predicate:((d_qoy = 1) and (d_year = 1998) and d_date_sk is not null) - TableScan [TS_66] (rows=73049 width=1119) - default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] + Group By Operator [GBY_78] (rows=22000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0 + <-Reducer 24 [SIMPLE_EDGE] + SHUFFLE [RS_77] + PartitionCols:_col0 + Group By Operator [GBY_76] (rows=44000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7 + Select Operator [SEL_75] (rows=44000000 width=1014) + Output:["_col7","_col2"] + Merge Join Operator [MERGEJOIN_269] (rows=44000000 width=1014) + Conds:RS_72._col1=RS_73._col0(Inner),Output:["_col2","_col7"] + <-Map 27 [SIMPLE_EDGE] + SHUFFLE [RS_73] + PartitionCols:_col0 + Select Operator [SEL_68] (rows=40000000 width=1014) + Output:["_col0","_col1"] + Filter Operator [FIL_255] (rows=40000000 width=1014) + predicate:(ca_address_sk is not null and ca_county is not null) + TableScan [TS_66] (rows=40000000 width=1014) + default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] + <-Reducer 23 [SIMPLE_EDGE] + SHUFFLE [RS_72] + PartitionCols:_col1 + Merge Join Operator [MERGEJOIN_268] (rows=20088 width=1119) + Conds:RS_69._col0=RS_70._col0(Inner),Output:["_col1","_col2"] + <-Map 22 [SIMPLE_EDGE] + SHUFFLE [RS_69] + PartitionCols:_col0 + Select Operator [SEL_62] (rows=1 width=0) + Output:["_col0","_col1","_col2"] + Filter Operator [FIL_253] (rows=1 width=0) + predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null) + TableScan [TS_60] (rows=1 width=0) + default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"] + <-Map 26 [SIMPLE_EDGE] + SHUFFLE [RS_70] + PartitionCols:_col0 + Select Operator [SEL_65] (rows=18262 width=1119) + Output:["_col0"] + Filter Operator [FIL_254] (rows=18262 width=1119) + predicate:((d_qoy = 1) and (d_year = 1998) and d_date_sk is not null) + TableScan [TS_63] (rows=73049 width=1119) + default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] <-Reducer 31 [SIMPLE_EDGE] - SHUFFLE [RS_109] + SHUFFLE [RS_104] PartitionCols:_col0 - Select Operator [SEL_104] (rows=22000000 width=1014) - Output:["_col0","_col3"] - Group By Operator [GBY_103] (rows=22000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, 2, 1998 - <-Reducer 30 [SIMPLE_EDGE] - SHUFFLE [RS_102] - PartitionCols:_col0, 2, 1998 - Group By Operator [GBY_101] (rows=44000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:_col0, 2, 1998 - Select Operator [SEL_99] (rows=44000000 width=1014) - Output:["_col0","_col3"] - Merge Join Operator [MERGEJOIN_277] (rows=44000000 width=1014) - Conds:RS_96._col1=RS_97._col0(Inner),Output:["_col2","_col7"] - <-Map 33 [SIMPLE_EDGE] - SHUFFLE [RS_97] - PartitionCols:_col0 - Select Operator [SEL_92] (rows=40000000 width=1014) - Output:["_col0","_col1"] - Filter Operator [FIL_264] (rows=40000000 width=1014) - predicate:(ca_address_sk is not null and ca_county is not null) - TableScan [TS_90] (rows=40000000 width=1014) - default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] - <-Reducer 29 [SIMPLE_EDGE] - SHUFFLE [RS_96] - PartitionCols:_col1 - Merge Join Operator [MERGEJOIN_276] (rows=20088 width=1119) - Conds:RS_93._col0=RS_94._col0(Inner),Output:["_col1","_col2"] - <-Map 28 [SIMPLE_EDGE] - SHUFFLE [RS_93] - PartitionCols:_col0 - Select Operator [SEL_86] (rows=1 width=0) - Output:["_col0","_col1","_col2"] - Filter Operator [FIL_262] (rows=1 width=0) - predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null) - TableScan [TS_84] (rows=1 width=0) - default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"] - <-Map 32 [SIMPLE_EDGE] - SHUFFLE [RS_94] - PartitionCols:_col0 - Select Operator [SEL_89] (rows=18262 width=1119) - Output:["_col0"] - Filter Operator [FIL_263] (rows=18262 width=1119) - predicate:((d_qoy = 2) and (d_year = 1998) and d_date_sk is not null) - TableScan [TS_87] (rows=73049 width=1119) - default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] + Group By Operator [GBY_98] (rows=22000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0 + <-Reducer 30 [SIMPLE_EDGE] + SHUFFLE [RS_97] + PartitionCols:_col0 + Group By Operator [GBY_96] (rows=44000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7 + Select Operator [SEL_95] (rows=44000000 width=1014) + Output:["_col7","_col2"] + Merge Join Operator [MERGEJOIN_271] (rows=44000000 width=1014) + Conds:RS_92._col1=RS_93._col0(Inner),Output:["_col2","_col7"] + <-Map 33 [SIMPLE_EDGE] + SHUFFLE [RS_93] + PartitionCols:_col0 + Select Operator [SEL_88] (rows=40000000 width=1014) + Output:["_col0","_col1"] + Filter Operator [FIL_258] (rows=40000000 width=1014) + predicate:(ca_address_sk is not null and ca_county is not null) + TableScan [TS_86] (rows=40000000 width=1014) + default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] + <-Reducer 29 [SIMPLE_EDGE] + SHUFFLE [RS_92] + PartitionCols:_col1 + Merge Join Operator [MERGEJOIN_270] (rows=20088 width=1119) + Conds:RS_89._col0=RS_90._col0(Inner),Output:["_col1","_col2"] + <-Map 28 [SIMPLE_EDGE] + SHUFFLE [RS_89] + PartitionCols:_col0 + Select Operator [SEL_82] (rows=1 width=0) + Output:["_col0","_col1","_col2"] + Filter Operator [FIL_256] (rows=1 width=0) + predicate:(ws_sold_date_sk is not null and ws_bill_addr_sk is not null) + TableScan [TS_80] (rows=1 width=0) + default@web_sales,web_sales,Tbl:PARTIAL,Col:NONE,Output:["ws_sold_date_sk","ws_bill_addr_sk","ws_ext_sales_price"] + <-Map 32 [SIMPLE_EDGE] + SHUFFLE [RS_90] + PartitionCols:_col0 + Select Operator [SEL_85] (rows=18262 width=1119) + Output:["_col0"] + Filter Operator [FIL_257] (rows=18262 width=1119) + predicate:((d_qoy = 2) and (d_year = 1998) and d_date_sk is not null) + TableScan [TS_83] (rows=73049 width=1119) + default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] <-Reducer 4 [SIMPLE_EDGE] - SHUFFLE [RS_105] + SHUFFLE [RS_100] PartitionCols:_col0 - Select Operator [SEL_20] (rows=22000000 width=1014) - Output:["_col0","_col3"] - Group By Operator [GBY_19] (rows=22000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, 1, 1998 + Select Operator [SEL_19] (rows=22000000 width=1014) + Output:["_col0","_col1","_col2"] + Group By Operator [GBY_18] (rows=22000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0 <-Reducer 3 [SIMPLE_EDGE] - SHUFFLE [RS_18] - PartitionCols:_col0, 1, 1998 - Group By Operator [GBY_17] (rows=44000000 width=1014) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:_col0, 1, 1998 + SHUFFLE [RS_17] + PartitionCols:_col0 + Group By Operator [GBY_16] (rows=44000000 width=1014) + Output:["_col0","_col1"],aggregations:["sum(_col2)"],keys:_col7 Select Operator [SEL_15] (rows=44000000 width=1014) - Output:["_col0","_col3"] - Merge Join Operator [MERGEJOIN_269] (rows=44000000 width=1014) + Output:["_col7","_col2"] + Merge Join Operator [MERGEJOIN_263] (rows=44000000 width=1014) Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col2","_col7"] <-Map 9 [SIMPLE_EDGE] SHUFFLE [RS_13] PartitionCols:_col0 Select Operator [SEL_8] (rows=40000000 width=1014) Output:["_col0","_col1"] - Filter Operator [FIL_252] (rows=40000000 width=1014) + Filter Operator [FIL_246] (rows=40000000 width=1014) predicate:(ca_address_sk is not null and ca_county is not null) TableScan [TS_6] (rows=40000000 width=1014) default@customer_address,customer_address,Tbl:COMPLETE,Col:NONE,Output:["ca_address_sk","ca_county"] <-Reducer 2 [SIMPLE_EDGE] SHUFFLE [RS_12] PartitionCols:_col1 - Merge Join Operator [MERGEJOIN_268] (rows=20088 width=1119) + Merge Join Operator [MERGEJOIN_262] (rows=20088 width=1119) Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col1","_col2"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_9] PartitionCols:_col0 Select Operator [SEL_2] (rows=1 width=0) Output:["_col0","_col1","_col2"] - Filter Operator [FIL_250] (rows=1 width=0) + Filter Operator [FIL_244] (rows=1 width=0) predicate:(ss_sold_date_sk is not null and ss_addr_sk is not null) TableScan [TS_0] (rows=1 width=0) default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_addr_sk","ss_ext_sales_price"] @@ -336,7 +326,7 @@ Stage-0 PartitionCols:_col0 Select Operator [SEL_5] (rows=18262 width=1119) Output:["_col0"] - Filter Operator [FIL_251] (rows=18262 width=1119) + Filter Operator [FIL_245] (rows=18262 width=1119) predicate:((d_qoy = 1) and (d_year = 1998) and d_date_sk is not null) TableScan [TS_3] (rows=73049 width=1119) default@date_dim,date_dim,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_qoy"] diff --git ql/src/test/results/clientpositive/perf/query39.q.out ql/src/test/results/clientpositive/perf/query39.q.out index 52c6b7c..e63aeec 100644 --- ql/src/test/results/clientpositive/perf/query39.q.out +++ ql/src/test/results/clientpositive/perf/query39.q.out @@ -27,28 +27,28 @@ Stage-0 <-Reducer 6 [SIMPLE_EDGE] SHUFFLE [RS_60] Select Operator [SEL_59] (rows=169103 width=1436) - Output:["_col0","_col1","_col3","_col4","_col5","_col6","_col8","_col9"] - Merge Join Operator [MERGEJOIN_105] (rows=169103 width=1436) - Conds:RS_56._col2, _col1=RS_57._col2, _col1(Inner),Output:["_col1","_col2","_col4","_col5","_col7","_col8","_col10","_col11"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] + Merge Join Operator [MERGEJOIN_103] (rows=169103 width=1436) + Conds:RS_56._col2, _col1=RS_57._col2, _col1(Inner),Output:["_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11"] <-Reducer 15 [SIMPLE_EDGE] SHUFFLE [RS_57] PartitionCols:_col2, _col1 Select Operator [SEL_55] (rows=153730 width=1436) - Output:["_col1","_col2","_col4","_col5"] + Output:["_col1","_col2","_col3","_col4","_col5"] Filter Operator [FIL_54] (rows=153730 width=1436) - predicate:CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col5 / _col4) > 1.0)) END - Select Operator [SEL_97] (rows=307461 width=1436) - Output:["_col1","_col2","_col4","_col5"] - Group By Operator [GBY_53] (rows=307461 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["avg(VALUE._col0)","stddev_samp(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, 4 + predicate:CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col3 / _col4) > 1.0)) END + Select Operator [SEL_53] (rows=307461 width=1436) + Output:["_col1","_col2","_col3","_col4"] + Group By Operator [GBY_52] (rows=307461 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["stddev_samp(VALUE._col0)","avg(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2 <-Reducer 14 [SIMPLE_EDGE] - SHUFFLE [RS_52] - PartitionCols:_col0, _col1, _col2, 4 - Group By Operator [GBY_51] (rows=614922 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["avg(_col4)","stddev_samp(_col4)"],keys:_col0, _col1, _col2, 4 + SHUFFLE [RS_51] + PartitionCols:_col0, _col1, _col2 + Group By Operator [GBY_50] (rows=614922 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["stddev_samp(_col3)","avg(_col3)"],keys:_col4, _col5, _col6 Select Operator [SEL_49] (rows=614922 width=1436) - Output:["_col0","_col1","_col2","_col4"] - Merge Join Operator [MERGEJOIN_104] (rows=614922 width=1436) + Output:["_col4","_col5","_col6","_col3"] + Merge Join Operator [MERGEJOIN_102] (rows=614922 width=1436) Conds:RS_46._col0=RS_47._col0(Inner),Output:["_col3","_col4","_col5","_col6"] <-Map 18 [SIMPLE_EDGE] SHUFFLE [RS_47] @@ -62,7 +62,7 @@ Stage-0 <-Reducer 13 [SIMPLE_EDGE] SHUFFLE [RS_46] PartitionCols:_col0 - Merge Join Operator [MERGEJOIN_103] (rows=559020 width=1436) + Merge Join Operator [MERGEJOIN_101] (rows=559020 width=1436) Conds:RS_43._col2=RS_44._col0(Inner),Output:["_col0","_col3","_col4","_col5","_col6"] <-Map 17 [SIMPLE_EDGE] SHUFFLE [RS_44] @@ -76,7 +76,7 @@ Stage-0 <-Reducer 12 [SIMPLE_EDGE] SHUFFLE [RS_43] PartitionCols:_col2 - Merge Join Operator [MERGEJOIN_102] (rows=508200 width=1436) + Merge Join Operator [MERGEJOIN_100] (rows=508200 width=1436) Conds:RS_40._col1=RS_41._col0(Inner),Output:["_col0","_col2","_col3","_col4"] <-Map 11 [SIMPLE_EDGE] SHUFFLE [RS_40] @@ -100,21 +100,21 @@ Stage-0 SHUFFLE [RS_56] PartitionCols:_col2, _col1 Select Operator [SEL_27] (rows=153730 width=1436) - Output:["_col1","_col2","_col4","_col5"] + Output:["_col1","_col2","_col3","_col4","_col5"] Filter Operator [FIL_26] (rows=153730 width=1436) - predicate:CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col5 / _col4) > 1.0)) END - Select Operator [SEL_98] (rows=307461 width=1436) - Output:["_col1","_col2","_col4","_col5"] - Group By Operator [GBY_25] (rows=307461 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["avg(VALUE._col0)","stddev_samp(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, 3 + predicate:CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col3 / _col4) > 1.0)) END + Select Operator [SEL_25] (rows=307461 width=1436) + Output:["_col1","_col2","_col3","_col4"] + Group By Operator [GBY_24] (rows=307461 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["stddev_samp(VALUE._col0)","avg(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2 <-Reducer 4 [SIMPLE_EDGE] - SHUFFLE [RS_24] - PartitionCols:_col0, _col1, _col2, 3 - Group By Operator [GBY_23] (rows=614922 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["avg(_col4)","stddev_samp(_col4)"],keys:_col0, _col1, _col2, 3 + SHUFFLE [RS_23] + PartitionCols:_col0, _col1, _col2 + Group By Operator [GBY_22] (rows=614922 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4"],aggregations:["stddev_samp(_col3)","avg(_col3)"],keys:_col4, _col5, _col6 Select Operator [SEL_21] (rows=614922 width=1436) - Output:["_col0","_col1","_col2","_col4"] - Merge Join Operator [MERGEJOIN_101] (rows=614922 width=1436) + Output:["_col4","_col5","_col6","_col3"] + Merge Join Operator [MERGEJOIN_99] (rows=614922 width=1436) Conds:RS_18._col0=RS_19._col0(Inner),Output:["_col3","_col4","_col5","_col6"] <-Map 10 [SIMPLE_EDGE] SHUFFLE [RS_19] @@ -128,7 +128,7 @@ Stage-0 <-Reducer 3 [SIMPLE_EDGE] SHUFFLE [RS_18] PartitionCols:_col0 - Merge Join Operator [MERGEJOIN_100] (rows=559020 width=1436) + Merge Join Operator [MERGEJOIN_98] (rows=559020 width=1436) Conds:RS_15._col2=RS_16._col0(Inner),Output:["_col0","_col3","_col4","_col5","_col6"] <-Map 9 [SIMPLE_EDGE] SHUFFLE [RS_16] @@ -142,7 +142,7 @@ Stage-0 <-Reducer 2 [SIMPLE_EDGE] SHUFFLE [RS_15] PartitionCols:_col2 - Merge Join Operator [MERGEJOIN_99] (rows=508200 width=1436) + Merge Join Operator [MERGEJOIN_97] (rows=508200 width=1436) Conds:RS_12._col1=RS_13._col0(Inner),Output:["_col0","_col2","_col3","_col4"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_12] diff --git ql/src/test/results/clientpositive/perf/query42.q.out ql/src/test/results/clientpositive/perf/query42.q.out index 1640d4e..c77f3c2 100644 --- ql/src/test/results/clientpositive/perf/query42.q.out +++ ql/src/test/results/clientpositive/perf/query42.q.out @@ -15,46 +15,46 @@ Stage-0 limit:100 Stage-1 Reducer 5 - File Output Operator [FS_24] - Limit [LIM_23] (rows=100 width=1436) + File Output Operator [FS_23] + Limit [LIM_22] (rows=100 width=1436) Number of rows:100 - Select Operator [SEL_22] (rows=127050 width=1436) + Select Operator [SEL_21] (rows=127050 width=1436) Output:["_col0","_col1","_col2","_col3"] <-Reducer 4 [SIMPLE_EDGE] - SHUFFLE [RS_21] - Select Operator [SEL_20] (rows=127050 width=1436) + SHUFFLE [RS_20] + Select Operator [SEL_19] (rows=127050 width=1436) Output:["_col1","_col2","_col3"] - Group By Operator [GBY_19] (rows=127050 width=1436) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:1998, KEY._col1, KEY._col2 + Group By Operator [GBY_18] (rows=127050 width=1436) + Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1 <-Reducer 3 [SIMPLE_EDGE] - SHUFFLE [RS_18] - PartitionCols:1998, _col1, _col2 - Group By Operator [GBY_17] (rows=254100 width=1436) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:1998, _col1, _col2 + SHUFFLE [RS_17] + PartitionCols:_col0, _col1 + Group By Operator [GBY_16] (rows=254100 width=1436) + Output:["_col0","_col1","_col2"],aggregations:["sum(_col5)"],keys:_col7, _col8 Select Operator [SEL_15] (rows=254100 width=1436) - Output:["_col1","_col2","_col3"] - Merge Join Operator [MERGEJOIN_34] (rows=254100 width=1436) + Output:["_col7","_col8","_col5"] + Merge Join Operator [MERGEJOIN_33] (rows=254100 width=1436) Conds:RS_12._col4=RS_13._col0(Inner),Output:["_col5","_col7","_col8"] <-Map 7 [SIMPLE_EDGE] SHUFFLE [RS_13] PartitionCols:_col0 Select Operator [SEL_8] (rows=231000 width=1436) Output:["_col0","_col1","_col2"] - Filter Operator [FIL_32] (rows=231000 width=1436) + Filter Operator [FIL_31] (rows=231000 width=1436) predicate:((i_manager_id = 1) and i_item_sk is not null) TableScan [TS_6] (rows=462000 width=1436) default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_category_id","i_category","i_manager_id"] <-Reducer 2 [SIMPLE_EDGE] SHUFFLE [RS_12] PartitionCols:_col4 - Merge Join Operator [MERGEJOIN_33] (rows=20088 width=1119) + Merge Join Operator [MERGEJOIN_32] (rows=20088 width=1119) Conds:RS_9._col0=RS_10._col0(Inner),Output:["_col4","_col5"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_9] PartitionCols:_col0 Select Operator [SEL_2] (rows=18262 width=1119) Output:["_col0"] - Filter Operator [FIL_30] (rows=18262 width=1119) + Filter Operator [FIL_29] (rows=18262 width=1119) predicate:((d_moy = 12) and (d_year = 1998) and d_date_sk is not null) TableScan [TS_0] (rows=73049 width=1119) default@date_dim,dt,Tbl:COMPLETE,Col:NONE,Output:["d_date_sk","d_year","d_moy"] @@ -63,7 +63,7 @@ Stage-0 PartitionCols:_col0 Select Operator [SEL_5] (rows=1 width=0) Output:["_col0","_col1","_col2"] - Filter Operator [FIL_31] (rows=1 width=0) + Filter Operator [FIL_30] (rows=1 width=0) predicate:(ss_sold_date_sk is not null and ss_item_sk is not null) TableScan [TS_3] (rows=1 width=0) default@store_sales,store_sales,Tbl:PARTIAL,Col:NONE,Output:["ss_sold_date_sk","ss_item_sk","ss_ext_sales_price"] diff --git ql/src/test/results/clientpositive/perf/query52.q.out ql/src/test/results/clientpositive/perf/query52.q.out index 7bf7317..a2b7652 100644 --- ql/src/test/results/clientpositive/perf/query52.q.out +++ ql/src/test/results/clientpositive/perf/query52.q.out @@ -22,17 +22,17 @@ Stage-0 Output:["_col0","_col1","_col2","_col3"] <-Reducer 4 [SIMPLE_EDGE] SHUFFLE [RS_21] - Select Operator [SEL_20] (rows=127050 width=1436) + Select Operator [SEL_19] (rows=127050 width=1436) Output:["_col1","_col2","_col3"] - Group By Operator [GBY_19] (rows=127050 width=1436) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:1998, KEY._col1, KEY._col2 + Group By Operator [GBY_18] (rows=127050 width=1436) + Output:["_col0","_col1","_col2"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1 <-Reducer 3 [SIMPLE_EDGE] - SHUFFLE [RS_18] - PartitionCols:1998, _col1, _col2 - Group By Operator [GBY_17] (rows=254100 width=1436) - Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(_col3)"],keys:1998, _col1, _col2 + SHUFFLE [RS_17] + PartitionCols:_col0, _col1 + Group By Operator [GBY_16] (rows=254100 width=1436) + Output:["_col0","_col1","_col2"],aggregations:["sum(_col5)"],keys:_col7, _col8 Select Operator [SEL_15] (rows=254100 width=1436) - Output:["_col1","_col2","_col3"] + Output:["_col7","_col8","_col5"] Merge Join Operator [MERGEJOIN_34] (rows=254100 width=1436) Conds:RS_12._col4=RS_13._col0(Inner),Output:["_col5","_col7","_col8"] <-Map 7 [SIMPLE_EDGE] diff --git ql/src/test/results/clientpositive/perf/query64.q.out ql/src/test/results/clientpositive/perf/query64.q.out index ff81523..a2d5b58 100644 --- ql/src/test/results/clientpositive/perf/query64.q.out +++ ql/src/test/results/clientpositive/perf/query64.q.out @@ -57,25 +57,25 @@ Stage-0 <-Reducer 19 [SIMPLE_EDGE] SHUFFLE [RS_253] Select Operator [SEL_252] (rows=122532649 width=860) - Output:["_col0","_col1","_col10","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col2","_col20","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20"] Filter Operator [FIL_251] (rows=122532649 width=860) - predicate:(_col34 <= _col15) + predicate:(_col32 <= _col14) Merge Join Operator [MERGEJOIN_716] (rows=367597947 width=860) - Conds:RS_248._col1, _col2, _col3=RS_249._col1, _col2, _col3(Inner),Output:["_col0","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col15","_col16","_col17","_col18","_col34","_col35","_col36","_col37"] + Conds:RS_248._col1, _col2, _col3=RS_249._col1, _col2, _col3(Inner),Output:["_col0","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col14","_col15","_col16","_col17","_col30","_col32","_col33","_col34","_col35"] <-Reducer 18 [SIMPLE_EDGE] SHUFFLE [RS_248] PartitionCols:_col1, _col2, _col3 - Select Operator [SEL_123] (rows=334179945 width=860) - Output:["_col0","_col1","_col10","_col11","_col15","_col16","_col17","_col18","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] - Group By Operator [GBY_122] (rows=334179945 width=860) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, 2000, KEY._col13, KEY._col14 + Select Operator [SEL_122] (rows=334179945 width=860) + Output:["_col0","_col1","_col10","_col11","_col12","_col14","_col15","_col16","_col17","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9"] + Group By Operator [GBY_121] (rows=334179945 width=860) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, KEY._col12, KEY._col13 <-Reducer 17 [SIMPLE_EDGE] - SHUFFLE [RS_121] - PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, 2000, _col13, _col14 - Group By Operator [GBY_120] (rows=668359891 width=860) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"],aggregations:["count()","sum(_col15)","sum(_col16)","sum(_col17)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, 2000, _col13, _col14 + SHUFFLE [RS_120] + PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13 + Group By Operator [GBY_119] (rows=668359891 width=860) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count()","sum(_col9)","sum(_col10)","sum(_col11)"],keys:_col23, _col25, _col27, _col28, _col39, _col40, _col41, _col42, _col44, _col45, _col46, _col47, _col50, _col53 Select Operator [SEL_118] (rows=668359891 width=860) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col13","_col14","_col15","_col16","_col17"] + Output:["_col23","_col25","_col27","_col28","_col39","_col40","_col41","_col42","_col44","_col45","_col46","_col47","_col50","_col53","_col9","_col10","_col11"] Merge Join Operator [MERGEJOIN_698] (rows=668359891 width=860) Conds:RS_114._col1=RS_115._col0(Inner),RS_114._col1=RS_116._col0(Inner),Output:["_col9","_col10","_col11","_col23","_col25","_col27","_col28","_col39","_col40","_col41","_col42","_col44","_col45","_col46","_col47","_col50","_col53"] <-Map 36 [SIMPLE_EDGE] @@ -349,17 +349,17 @@ Stage-0 <-Reducer 58 [SIMPLE_EDGE] SHUFFLE [RS_249] PartitionCols:_col1, _col2, _col3 - Select Operator [SEL_247] (rows=334179945 width=860) - Output:["_col1","_col15","_col16","_col17","_col18","_col2","_col3"] - Group By Operator [GBY_246] (rows=334179945 width=860) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, 2001, KEY._col13, KEY._col14 + Select Operator [SEL_246] (rows=334179945 width=860) + Output:["_col1","_col12","_col14","_col15","_col16","_col17","_col2","_col3"] + Group By Operator [GBY_245] (rows=334179945 width=860) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7, KEY._col8, KEY._col9, KEY._col10, KEY._col11, KEY._col12, KEY._col13 <-Reducer 57 [SIMPLE_EDGE] - SHUFFLE [RS_245] - PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, 2001, _col13, _col14 - Group By Operator [GBY_244] (rows=668359891 width=860) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18"],aggregations:["count()","sum(_col15)","sum(_col16)","sum(_col17)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, 2001, _col13, _col14 + SHUFFLE [RS_244] + PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13 + Group By Operator [GBY_243] (rows=668359891 width=860) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17"],aggregations:["count()","sum(_col9)","sum(_col10)","sum(_col11)"],keys:_col23, _col25, _col27, _col28, _col39, _col40, _col41, _col42, _col44, _col45, _col46, _col47, _col50, _col53 Select Operator [SEL_242] (rows=668359891 width=860) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col13","_col14","_col15","_col16","_col17"] + Output:["_col23","_col25","_col27","_col28","_col39","_col40","_col41","_col42","_col44","_col45","_col46","_col47","_col50","_col53","_col9","_col10","_col11"] Merge Join Operator [MERGEJOIN_715] (rows=668359891 width=860) Conds:RS_238._col1=RS_239._col0(Inner),RS_238._col1=RS_240._col0(Inner),Output:["_col9","_col10","_col11","_col23","_col25","_col27","_col28","_col39","_col40","_col41","_col42","_col44","_col45","_col46","_col47","_col50","_col53"] <-Map 74 [SIMPLE_EDGE] diff --git ql/src/test/results/clientpositive/perf/query66.q.out ql/src/test/results/clientpositive/perf/query66.q.out index d698602..7478594 100644 --- ql/src/test/results/clientpositive/perf/query66.q.out +++ ql/src/test/results/clientpositive/perf/query66.q.out @@ -464,27 +464,27 @@ Stage-0 Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"] <-Reducer 8 [SIMPLE_EDGE] SHUFFLE [RS_73] - Group By Operator [GBY_71] (rows=26136 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)","sum(VALUE._col24)","sum(VALUE._col25)","sum(VALUE._col26)","sum(VALUE._col27)","sum(VALUE._col28)","sum(VALUE._col29)","sum(VALUE._col30)","sum(VALUE._col31)","sum(VALUE._col32)","sum(VALUE._col33)","sum(VALUE._col34)","sum(VALUE._col35)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, KEY._col6, KEY._col7 - <-Union 7 [SIMPLE_EDGE] - <-Reducer 19 [CONTAINS] - Reduce Output Operator [RS_70] - PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Group By Operator [GBY_69] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Select Operator [SEL_67] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"] - Select Operator [SEL_65] (rows=26136 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31"] + Select Operator [SEL_72] (rows=26136 width=471) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"] + Group By Operator [GBY_71] (rows=26136 width=471) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)","sum(VALUE._col24)","sum(VALUE._col25)","sum(VALUE._col26)","sum(VALUE._col27)","sum(VALUE._col28)","sum(VALUE._col29)","sum(VALUE._col30)","sum(VALUE._col31)","sum(VALUE._col32)","sum(VALUE._col33)","sum(VALUE._col34)","sum(VALUE._col35)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5 + <-Union 7 [SIMPLE_EDGE] + <-Reducer 19 [CONTAINS] + Reduce Output Operator [RS_70] + PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5 + Group By Operator [GBY_69] (rows=52272 width=471) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)"],keys:_col0, _col1, _col2, _col3, _col4, _col5 + Select Operator [SEL_67] (rows=52272 width=471) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41"] Group By Operator [GBY_64] (rows=26136 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002 + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5 <-Reducer 18 [SIMPLE_EDGE] SHUFFLE [RS_63] - PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002 + PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5 Group By Operator [GBY_62] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002 + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)"],keys:_col0, _col1, _col2, _col3, _col4, _col5 Select Operator [SEL_60] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"] Merge Join Operator [MERGEJOIN_122] (rows=52272 width=471) Conds:RS_57._col2=RS_58._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"] <-Map 23 [SIMPLE_EDGE] @@ -547,24 +547,22 @@ Stage-0 predicate:w_warehouse_sk is not null TableScan [TS_36] (rows=27 width=1029) default@warehouse,warehouse,Tbl:COMPLETE,Col:NONE,Output:["w_warehouse_sk","w_warehouse_name","w_warehouse_sq_ft","w_city","w_county","w_state","w_country"] - <-Reducer 6 [CONTAINS] - Reduce Output Operator [RS_70] - PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Group By Operator [GBY_69] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"],aggregations:["sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)","sum(_col42)","sum(_col43)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 - Select Operator [SEL_67] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41","_col42","_col43"] - Select Operator [SEL_32] (rows=26136 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31"] + <-Reducer 6 [CONTAINS] + Reduce Output Operator [RS_70] + PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5 + Group By Operator [GBY_69] (rows=52272 width=471) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)","sum(_col31)","sum(_col32)","sum(_col33)","sum(_col34)","sum(_col35)","sum(_col36)","sum(_col37)","sum(_col38)","sum(_col39)","sum(_col40)","sum(_col41)"],keys:_col0, _col1, _col2, _col3, _col4, _col5 + Select Operator [SEL_67] (rows=52272 width=471) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30","_col31","_col32","_col33","_col34","_col35","_col36","_col37","_col38","_col39","_col40","_col41"] Group By Operator [GBY_31] (rows=26136 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5, 2002 + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)","sum(VALUE._col2)","sum(VALUE._col3)","sum(VALUE._col4)","sum(VALUE._col5)","sum(VALUE._col6)","sum(VALUE._col7)","sum(VALUE._col8)","sum(VALUE._col9)","sum(VALUE._col10)","sum(VALUE._col11)","sum(VALUE._col12)","sum(VALUE._col13)","sum(VALUE._col14)","sum(VALUE._col15)","sum(VALUE._col16)","sum(VALUE._col17)","sum(VALUE._col18)","sum(VALUE._col19)","sum(VALUE._col20)","sum(VALUE._col21)","sum(VALUE._col22)","sum(VALUE._col23)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4, KEY._col5 <-Reducer 5 [SIMPLE_EDGE] SHUFFLE [RS_30] - PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5, 2002 + PartitionCols:_col0, _col1, _col2, _col3, _col4, _col5 Group By Operator [GBY_29] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"],aggregations:["sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)","sum(_col30)"],keys:_col0, _col1, _col2, _col3, _col4, _col5, 2002 + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"],aggregations:["sum(_col6)","sum(_col7)","sum(_col8)","sum(_col9)","sum(_col10)","sum(_col11)","sum(_col12)","sum(_col13)","sum(_col14)","sum(_col15)","sum(_col16)","sum(_col17)","sum(_col18)","sum(_col19)","sum(_col20)","sum(_col21)","sum(_col22)","sum(_col23)","sum(_col24)","sum(_col25)","sum(_col26)","sum(_col27)","sum(_col28)","sum(_col29)"],keys:_col0, _col1, _col2, _col3, _col4, _col5 Select Operator [SEL_27] (rows=52272 width=471) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29","_col30"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6","_col7","_col8","_col9","_col10","_col11","_col12","_col13","_col14","_col15","_col16","_col17","_col18","_col19","_col20","_col21","_col22","_col23","_col24","_col25","_col26","_col27","_col28","_col29"] Merge Join Operator [MERGEJOIN_118] (rows=52272 width=471) Conds:RS_24._col2=RS_25._col0(Inner),Output:["_col4","_col5","_col6","_col8","_col9","_col10","_col11","_col12","_col13","_col16"] <-Map 13 [SIMPLE_EDGE] diff --git ql/src/test/results/clientpositive/perf/query75.q.out ql/src/test/results/clientpositive/perf/query75.q.out index 15c46c2..caf49ba 100644 --- ql/src/test/results/clientpositive/perf/query75.q.out +++ ql/src/test/results/clientpositive/perf/query75.q.out @@ -49,18 +49,18 @@ Stage-0 <-Reducer 31 [SIMPLE_EDGE] SHUFFLE [RS_149] PartitionCols:_col1, _col2, _col3, _col4 - Group By Operator [GBY_146] (rows=461191 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4 - <-Union 30 [SIMPLE_EDGE] - <-Reducer 29 [CONTAINS] - Reduce Output Operator [RS_145] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_144] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4 - Select Operator [SEL_142] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + Select Operator [SEL_147] (rows=461191 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + Group By Operator [GBY_146] (rows=461191 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3 + <-Union 30 [SIMPLE_EDGE] + <-Reducer 29 [CONTAINS] + Reduce Output Operator [RS_145] + PartitionCols:_col0, _col1, _col2, _col3 + Group By Operator [GBY_144] (rows=922383 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3 Select Operator [SEL_95] (rows=307461 width=1436) - Output:["_col1","_col2","_col3","_col4","_col5","_col6"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] Merge Join Operator [MERGEJOIN_252] (rows=307461 width=1436) Conds:RS_92._col2, _col1=RS_93._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"] <-Map 34 [SIMPLE_EDGE] @@ -106,18 +106,16 @@ Stage-0 Select Operator [SEL_79] (rows=231000 width=1436) Output:["_col0","_col1","_col2","_col3","_col5"] Filter Operator [FIL_230] (rows=231000 width=1436) - predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null) + predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null) TableScan [TS_77] (rows=462000 width=1436) default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"] - <-Reducer 38 [CONTAINS] - Reduce Output Operator [RS_145] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_144] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4 - Select Operator [SEL_142] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + <-Reducer 38 [CONTAINS] + Reduce Output Operator [RS_145] + PartitionCols:_col0, _col1, _col2, _col3 + Group By Operator [GBY_144] (rows=922383 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3 Select Operator [SEL_117] (rows=307461 width=1436) - Output:["_col1","_col2","_col3","_col4","_col5","_col6"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] Merge Join Operator [MERGEJOIN_255] (rows=307461 width=1436) Conds:RS_114._col2, _col1=RS_115._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"] <-Map 41 [SIMPLE_EDGE] @@ -163,18 +161,16 @@ Stage-0 Select Operator [SEL_101] (rows=231000 width=1436) Output:["_col0","_col1","_col2","_col3","_col5"] Filter Operator [FIL_234] (rows=231000 width=1436) - predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null) + predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null) TableScan [TS_99] (rows=462000 width=1436) default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"] - <-Reducer 45 [CONTAINS] - Reduce Output Operator [RS_145] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_144] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4 - Select Operator [SEL_142] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + <-Reducer 45 [CONTAINS] + Reduce Output Operator [RS_145] + PartitionCols:_col0, _col1, _col2, _col3 + Group By Operator [GBY_144] (rows=922383 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3 Select Operator [SEL_141] (rows=307461 width=1436) - Output:["_col1","_col2","_col3","_col4","_col5","_col6"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] Merge Join Operator [MERGEJOIN_258] (rows=307461 width=1436) Conds:RS_138._col2, _col1=RS_139._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"] <-Map 48 [SIMPLE_EDGE] @@ -220,24 +216,24 @@ Stage-0 Select Operator [SEL_125] (rows=231000 width=1436) Output:["_col0","_col1","_col2","_col3","_col5"] Filter Operator [FIL_238] (rows=231000 width=1436) - predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null) + predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null) TableScan [TS_123] (rows=462000 width=1436) default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"] <-Reducer 6 [SIMPLE_EDGE] SHUFFLE [RS_148] PartitionCols:_col1, _col2, _col3, _col4 - Group By Operator [GBY_72] (rows=461191 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3, KEY._col4 - <-Union 5 [SIMPLE_EDGE] - <-Reducer 15 [CONTAINS] - Reduce Output Operator [RS_71] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_70] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4 - Select Operator [SEL_68] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + Select Operator [SEL_73] (rows=461191 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + Group By Operator [GBY_72] (rows=461191 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(VALUE._col0)","sum(VALUE._col1)"],keys:KEY._col0, KEY._col1, KEY._col2, KEY._col3 + <-Union 5 [SIMPLE_EDGE] + <-Reducer 15 [CONTAINS] + Reduce Output Operator [RS_71] + PartitionCols:_col0, _col1, _col2, _col3 + Group By Operator [GBY_70] (rows=922383 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3 Select Operator [SEL_43] (rows=307461 width=1436) - Output:["_col1","_col2","_col3","_col4","_col5","_col6"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] Merge Join Operator [MERGEJOIN_246] (rows=307461 width=1436) Conds:RS_40._col2, _col1=RS_41._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"] <-Map 18 [SIMPLE_EDGE] @@ -283,18 +279,16 @@ Stage-0 Select Operator [SEL_27] (rows=231000 width=1436) Output:["_col0","_col1","_col2","_col3","_col5"] Filter Operator [FIL_222] (rows=231000 width=1436) - predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null) + predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null) TableScan [TS_25] (rows=462000 width=1436) default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"] - <-Reducer 22 [CONTAINS] - Reduce Output Operator [RS_71] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_70] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4 - Select Operator [SEL_68] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + <-Reducer 22 [CONTAINS] + Reduce Output Operator [RS_71] + PartitionCols:_col0, _col1, _col2, _col3 + Group By Operator [GBY_70] (rows=922383 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3 Select Operator [SEL_67] (rows=307461 width=1436) - Output:["_col1","_col2","_col3","_col4","_col5","_col6"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] Merge Join Operator [MERGEJOIN_249] (rows=307461 width=1436) Conds:RS_64._col2, _col1=RS_65._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"] <-Map 25 [SIMPLE_EDGE] @@ -340,18 +334,16 @@ Stage-0 Select Operator [SEL_51] (rows=231000 width=1436) Output:["_col0","_col1","_col2","_col3","_col5"] Filter Operator [FIL_226] (rows=231000 width=1436) - predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null) + predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null) TableScan [TS_49] (rows=462000 width=1436) default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"] - <-Reducer 4 [CONTAINS] - Reduce Output Operator [RS_71] - PartitionCols:_col0, _col1, _col2, _col3, _col4 - Group By Operator [GBY_70] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"],aggregations:["sum(_col5)","sum(_col6)"],keys:_col0, _col1, _col2, _col3, _col4 - Select Operator [SEL_68] (rows=922383 width=1436) - Output:["_col0","_col1","_col2","_col3","_col4","_col5","_col6"] + <-Reducer 4 [CONTAINS] + Reduce Output Operator [RS_71] + PartitionCols:_col0, _col1, _col2, _col3 + Group By Operator [GBY_70] (rows=922383 width=1436) + Output:["_col0","_col1","_col2","_col3","_col4","_col5"],aggregations:["sum(_col4)","sum(_col5)"],keys:_col0, _col1, _col2, _col3 Select Operator [SEL_21] (rows=307461 width=1436) - Output:["_col1","_col2","_col3","_col4","_col5","_col6"] + Output:["_col0","_col1","_col2","_col3","_col4","_col5"] Merge Join Operator [MERGEJOIN_243] (rows=307461 width=1436) Conds:RS_18._col2, _col1=RS_19._col1, _col0(Left Outer),Output:["_col3","_col4","_col6","_col7","_col8","_col10","_col15","_col16"] <-Map 11 [SIMPLE_EDGE] @@ -397,7 +389,7 @@ Stage-0 Select Operator [SEL_5] (rows=231000 width=1436) Output:["_col0","_col1","_col2","_col3","_col5"] Filter Operator [FIL_218] (rows=231000 width=1436) - predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null and i_manufact_id is not null) + predicate:((i_category = 'Sports') and i_item_sk is not null and i_brand_id is not null and i_class_id is not null and i_category_id is not null) TableScan [TS_3] (rows=462000 width=1436) default@item,item,Tbl:COMPLETE,Col:NONE,Output:["i_item_sk","i_brand_id","i_class_id","i_category_id","i_category","i_manufact_id"] diff --git ql/src/test/results/clientpositive/pointlookup2.q.out ql/src/test/results/clientpositive/pointlookup2.q.out index d0ad68a..5facf9a 100644 --- ql/src/test/results/clientpositive/pointlookup2.q.out +++ ql/src/test/results/clientpositive/pointlookup2.q.out @@ -379,23 +379,27 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: int), _col4 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string - escape.delim \ - serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4 + columns.types int,string,string,int,string + escape.delim \ + serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-2 Map Reduce @@ -408,7 +412,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col1 (type: string), _col3 (type: int), _col4 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -419,8 +423,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4 + columns.types int,string,string,int,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -428,8 +432,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4 + columns.types int,string,string,int,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -438,7 +442,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), '2000-04-08' (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col1 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -639,23 +643,27 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: int), _col4 (type: string), '2000-04-09' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string - escape.delim \ - serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4,_col5 + columns.types int,string,string,int,string,string + escape.delim \ + serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-2 Map Reduce @@ -668,7 +676,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col1 (type: string), _col3 (type: int), _col4 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string), _col5 (type: string) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -679,8 +687,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4,_col5 + columns.types int,string,string,int,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -688,8 +696,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4 - columns.types int,string,int,string + columns _col0,_col1,_col2,_col3,_col4,_col5 + columns.types int,string,string,int,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -698,7 +706,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), '2000-04-09' (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -730,7 +738,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain extended select * from pcr_t1 t1 join pcr_t2 t2 @@ -1032,7 +1040,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain extended select * from pcr_t1 t1 join pcr_t2 t2 diff --git ql/src/test/results/clientpositive/pointlookup3.q.out ql/src/test/results/clientpositive/pointlookup3.q.out index 39804cf..def1bc8 100644 --- ql/src/test/results/clientpositive/pointlookup3.q.out +++ ql/src/test/results/clientpositive/pointlookup3.q.out @@ -305,7 +305,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), '2001-04-08' (type: string) + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 10 Data size: 80 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -461,23 +461,27 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4, _col5, _col6 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: string), _col4 (type: int), _col5 (type: string), _col6 (type: string), '2001-04-08' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col3,_col4,_col5,_col6 - columns.types int,string,string,int,string,string - escape.delim \ - serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7 + columns.types int,string,string,string,int,string,string,string + escape.delim \ + serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-2 Map Reduce @@ -490,7 +494,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col1 (type: string), _col3 (type: string), _col4 (type: int), _col5 (type: string), _col6 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: int), _col5 (type: string), _col6 (type: string), _col7 (type: string) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -501,8 +505,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4,_col5,_col6 - columns.types int,string,string,int,string,string + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7 + columns.types int,string,string,string,int,string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -510,8 +514,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4,_col5,_col6 - columns.types int,string,string,int,string,string + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7 + columns.types int,string,string,string,int,string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -520,7 +524,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: string), '2001-04-08' (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: string), VALUE._col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -723,23 +727,27 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4, _col5, _col7 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: string), _col4 (type: int), _col5 (type: string), '2000-04-09' (type: string), _col7 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col3,_col4,_col5,_col7 - columns.types int,string,string,int,string,string - escape.delim \ - serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7 + columns.types int,string,string,string,int,string,string,string + escape.delim \ + serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Stage: Stage-2 Map Reduce @@ -752,7 +760,7 @@ STAGE PLANS: sort order: + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col1 (type: string), _col3 (type: string), _col4 (type: int), _col5 (type: string), _col7 (type: string) + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: int), _col5 (type: string), _col6 (type: string), _col7 (type: string) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -763,8 +771,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4,_col5,_col7 - columns.types int,string,string,int,string,string + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7 + columns.types int,string,string,string,int,string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -772,8 +780,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1,_col3,_col4,_col5,_col7 - columns.types int,string,string,int,string,string + columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7 + columns.types int,string,string,string,int,string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -782,7 +790,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), '2000-04-09' (type: string), VALUE._col6 (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: int), VALUE._col4 (type: string), VALUE._col5 (type: string), VALUE._col6 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -814,7 +822,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain extended select * from pcr_t1 t1 join pcr_t1 t2 diff --git ql/src/test/results/clientpositive/ppd_outer_join4.q.out ql/src/test/results/clientpositive/ppd_outer_join4.q.out index ba5d187..ea0772a 100644 --- ql/src/test/results/clientpositive/ppd_outer_join4.q.out +++ ql/src/test/results/clientpositive/ppd_outer_join4.q.out @@ -38,33 +38,33 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) + predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string) - outputColumnNames: _col0 + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) TableScan alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) + predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) TableScan alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE @@ -85,7 +85,7 @@ STAGE PLANS: Join Operator condition map: Inner Join 0 to 1 - Inner Join 1 to 2 + Inner Join 0 to 2 keys: 0 _col0 (type: string) 1 _col0 (type: string) @@ -93,7 +93,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col0 (type: string) + expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -402,33 +402,33 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) + predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string) - outputColumnNames: _col0 + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) TableScan alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) + predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) TableScan alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE @@ -449,7 +449,7 @@ STAGE PLANS: Join Operator condition map: Inner Join 0 to 1 - Inner Join 1 to 2 + Inner Join 0 to 2 keys: 0 _col0 (type: string) 1 _col0 (type: string) @@ -457,7 +457,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col0 (type: string) + expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/ppd_outer_join5.q.out ql/src/test/results/clientpositive/ppd_outer_join5.q.out index 65ca9d1..55e5f06 100644 --- ql/src/test/results/clientpositive/ppd_outer_join5.q.out +++ ql/src/test/results/clientpositive/ppd_outer_join5.q.out @@ -30,7 +30,7 @@ POSTHOOK: query: create table t4 (id int, key string, value string) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t4 -Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain select * from t1 full outer join t2 on t1.id=t2.id join t3 on t2.id=t3.id where t3.id=20 PREHOOK: type: QUERY POSTHOOK: query: explain select * from t1 full outer join t2 on t1.id=t2.id join t3 on t2.id=t3.id where t3.id=20 @@ -50,7 +50,7 @@ STAGE PLANS: predicate: (id = 20) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: 20 (type: int), key (type: string), value (type: string) + expressions: id (type: int), key (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator @@ -118,7 +118,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t2.id=t3.id) where t2.id=20 PREHOOK: type: QUERY POSTHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t2.id=t3.id) where t2.id=20 @@ -166,7 +166,7 @@ STAGE PLANS: predicate: (id = 20) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: 20 (type: int), key (type: string), value (type: string) + expressions: id (type: int), key (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator @@ -202,7 +202,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t1.id=t3.id) where t2.id=20 PREHOOK: type: QUERY POSTHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t1.id=t3.id) where t2.id=20 @@ -250,7 +250,7 @@ STAGE PLANS: predicate: (id = 20) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: 20 (type: int), key (type: string), value (type: string) + expressions: id (type: int), key (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator diff --git ql/src/test/results/clientpositive/ppd_udf_case.q.out ql/src/test/results/clientpositive/ppd_udf_case.q.out index 1c1c2a4..0afc645 100644 --- ql/src/test/results/clientpositive/ppd_udf_case.q.out +++ ql/src/test/results/clientpositive/ppd_udf_case.q.out @@ -91,7 +91,7 @@ STAGE PLANS: Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey7 (type: string) + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -237,7 +237,7 @@ STAGE PLANS: Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey7 (type: string) + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey3 (type: string), KEY.reducesinkkey4 (type: string), KEY.reducesinkkey5 (type: string), KEY.reducesinkkey2 (type: string), KEY.reducesinkkey7 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7 Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/ppd_union_view.q.out ql/src/test/results/clientpositive/ppd_union_view.q.out index 435b6f9..76daa06 100644 --- ql/src/test/results/clientpositive/ppd_union_view.q.out +++ ql/src/test/results/clientpositive/ppd_union_view.q.out @@ -156,14 +156,14 @@ STAGE PLANS: predicate: keymap is not null (type: boolean) Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: keymap (type: string), value (type: string), ds (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: keymap (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), _col2 (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: _col0 (type: string), _col2 (type: string) + key expressions: _col0 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 14 Basic stats: COMPLETE Column stats: NONE tag: 0 value expressions: _col1 (type: string) @@ -177,14 +177,14 @@ STAGE PLANS: predicate: keymap is not null (type: boolean) Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), keymap (type: string), ds (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: key (type: string), keymap (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col1 (type: string), _col2 (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: _col1 (type: string), _col2 (type: string) + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE tag: 1 value expressions: _col0 (type: string) @@ -291,13 +291,13 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 _col0 (type: string), _col2 (type: string) - 1 _col1 (type: string), _col2 (type: string) + 0 _col0 (type: string) + 1 _col1 (type: string) outputColumnNames: _col1, _col3 Statistics: Num rows: 1 Data size: 15 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col3 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 + expressions: _col3 (type: string), _col1 (type: string), '2011-10-13' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 15 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false @@ -308,8 +308,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1 - columns.types string,string + columns _col0,_col1,_col2 + columns.types string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -329,67 +329,59 @@ STAGE PLANS: predicate: (ds = '2011-10-13') (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string), value (type: string), '2011-10-13' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '2011-10-13' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE -#### A masked pattern was here #### - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col2 - columns.types string:string:string - escape.delim \ - hive.serialization.extend.additional.nesting.levels true - serialization.escape.crlf true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types string:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.escape.crlf true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false TableScan GatherStats: false Union Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '2011-10-13' (type: string) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE + NumFilesPerFileSink: 1 + Statistics: Num rows: 2 Data size: 15 Basic stats: COMPLETE Column stats: NONE #### A masked pattern was here #### - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col2 - columns.types string:string:string - escape.delim \ - hive.serialization.extend.additional.nesting.levels true - serialization.escape.crlf true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types string:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.escape.crlf true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -399,8 +391,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1 - columns.types string,string + columns _col0,_col1,_col2 + columns.types string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -408,8 +400,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1 - columns.types string,string + columns _col0,_col1,_col2 + columns.types string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -488,10 +480,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), '2011-10-15' (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: _col0 (type: string), '2011-10-15' (type: string) + key expressions: _col0 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE tag: 0 value expressions: _col1 (type: string) @@ -509,10 +501,10 @@ STAGE PLANS: outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col1 (type: string), '2011-10-15' (type: string) - null sort order: aa - sort order: ++ - Map-reduce partition columns: _col1 (type: string), '2011-10-15' (type: string) + key expressions: _col1 (type: string) + null sort order: a + sort order: + + Map-reduce partition columns: _col1 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE tag: 1 value expressions: _col0 (type: string) @@ -523,13 +515,13 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 _col0 (type: string), _col2 (type: string) - 1 _col1 (type: string), _col2 (type: string) + 0 _col0 (type: string) + 1 _col1 (type: string) outputColumnNames: _col1, _col3 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col3 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 + expressions: _col3 (type: string), _col1 (type: string), '2011-10-15' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: false @@ -540,8 +532,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1 - columns.types string,string + columns _col0,_col1,_col2 + columns.types string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -557,45 +549,11 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE GatherStats: false Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string), value (type: string), '2011-10-15' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 2 Data size: 11 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '2011-10-15' (type: string) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 11 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 - Statistics: Num rows: 2 Data size: 11 Basic stats: COMPLETE Column stats: NONE -#### A masked pattern was here #### - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col2 - columns.types string:string:string - escape.delim \ - hive.serialization.extend.additional.nesting.levels true - serialization.escape.crlf true - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - TableScan - GatherStats: false - Union - Statistics: Num rows: 2 Data size: 11 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string), '2011-10-15' (type: string) - outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 2 Data size: 11 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false GlobalTableId: 0 @@ -618,6 +576,32 @@ STAGE PLANS: TotalFiles: 1 GatherStats: false MultiFileSpray: false + TableScan + GatherStats: false + Union + Statistics: Num rows: 2 Data size: 11 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + GlobalTableId: 0 +#### A masked pattern was here #### + NumFilesPerFileSink: 1 + Statistics: Num rows: 2 Data size: 11 Basic stats: COMPLETE Column stats: NONE +#### A masked pattern was here #### + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2 + columns.types string:string:string + escape.delim \ + hive.serialization.extend.additional.nesting.levels true + serialization.escape.crlf true + serialization.format 1 + serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Path -> Alias: #### A masked pattern was here #### Path -> Partition: @@ -627,8 +611,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1 - columns.types string,string + columns _col0,_col1,_col2 + columns.types string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe @@ -636,8 +620,8 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat properties: - columns _col0,_col1 - columns.types string,string + columns _col0,_col1,_col2 + columns.types string,string,string escape.delim \ serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe diff --git ql/src/test/results/clientpositive/quotedid_basic.q.out ql/src/test/results/clientpositive/quotedid_basic.q.out index 9ca3f6a..4f79a46 100644 --- ql/src/test/results/clientpositive/quotedid_basic.q.out +++ ql/src/test/results/clientpositive/quotedid_basic.q.out @@ -102,23 +102,23 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: x+1 (type: string), y&y (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: x+1, y&y Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator - keys: _col0 (type: string), _col1 (type: string), '1' (type: string) + keys: x+1 (type: string), y&y (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: string), '1' (type: string) - sort order: +++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: string), '1' (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Operator Tree: Group By Operator - keys: KEY._col0 (type: string), KEY._col1 (type: string), '1' (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string), '1' (type: string) @@ -161,34 +161,30 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: x+1 (type: string), y&y (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: x+1, y&y Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator - keys: _col0 (type: string), _col1 (type: string), '1' (type: string) + keys: x+1 (type: string), y&y (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: string), '1' (type: string) - sort order: +++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: string), '1' (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Operator Tree: Group By Operator - keys: KEY._col0 (type: string), KEY._col1 (type: string), '1' (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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 @@ -202,20 +198,20 @@ STAGE PLANS: value expressions: _col0 (type: string) Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE PTF Operator Function definitions: Input definition input alias: ptf_0 - output shape: _col0: string, _col1: string, _col2: string + output shape: _col0: string, _col1: string type: WINDOWING Windowing table definition input alias: ptf_1 name: windowingtablefunction order by: _col1 ASC NULLS FIRST - partition by: _col2 + partition by: '1' raw input shape: window functions: window function definition @@ -269,34 +265,30 @@ STAGE PLANS: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator expressions: x+1 (type: string), y&y (type: string) - outputColumnNames: _col0, _col1 + outputColumnNames: x+1, y&y Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator - keys: _col0 (type: string), _col1 (type: string), '1' (type: string) + keys: x+1 (type: string), y&y (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: string), '1' (type: string) - sort order: +++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: string), '1' (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Operator Tree: Group By Operator - keys: KEY._col0 (type: string), KEY._col1 (type: string), '1' (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: _col0 (type: string), _col1 (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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 @@ -310,20 +302,20 @@ STAGE PLANS: value expressions: _col0 (type: string) Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + expressions: VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE PTF Operator Function definitions: Input definition input alias: ptf_0 - output shape: _col0: string, _col1: string, _col2: string + output shape: _col0: string, _col1: string type: WINDOWING Windowing table definition input alias: ptf_1 name: windowingtablefunction order by: _col1 ASC NULLS FIRST - partition by: _col2 + partition by: '1' raw input shape: window functions: window function definition diff --git ql/src/test/results/clientpositive/quotedid_partition.q.out ql/src/test/results/clientpositive/quotedid_partition.q.out index 66cff2a..fe4c5db 100644 --- ql/src/test/results/clientpositive/quotedid_partition.q.out +++ ql/src/test/results/clientpositive/quotedid_partition.q.out @@ -47,26 +47,26 @@ STAGE PLANS: Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: y&y (type: string) - outputColumnNames: _col1 + outputColumnNames: y&y Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: '10' (type: string), _col1 (type: string), 'a' (type: string) + keys: y&y (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '10' (type: string), _col1 (type: string), 'a' (type: string) - sort order: +++ - Map-reduce partition columns: '10' (type: string), _col1 (type: string), 'a' (type: string) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Group By Operator - keys: '10' (type: string), KEY._col1 (type: string), 'a' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0 Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: '10' (type: string), _col1 (type: string), 'a' (type: string) + expressions: '10' (type: string), _col0 (type: string), 'a' (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/rand_partitionpruner3.q.out ql/src/test/results/clientpositive/rand_partitionpruner3.q.out index a2a06b2..eabf9d9 100644 --- ql/src/test/results/clientpositive/rand_partitionpruner3.q.out +++ ql/src/test/results/clientpositive/rand_partitionpruner3.q.out @@ -65,12 +65,12 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: ((rand(1) < 0.1) and ((UDFToDouble(key) <= 50.0) and (UDFToDouble(key) >= 10.0))) (type: boolean) - Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE + predicate: ((rand(1) < 0.1) and (not ((UDFToDouble(key) > 50.0) or (UDFToDouble(key) < 10.0)))) (type: boolean) + Statistics: Num rows: 56 Data size: 594 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string), '2008-04-08' (type: string), hr (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 18 Data size: 191 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 56 Data size: 594 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select a.* from srcpart a where rand(1) < 0.1 and a.ds = '2008-04-08' and not(key > 50 or key < 10) and a.hr like '%2' @@ -153,12 +153,12 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: ((UDFToDouble(key) <= 50.0) and (UDFToDouble(key) >= 10.0)) (type: boolean) - Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE + predicate: (not ((UDFToDouble(key) > 50.0) or (UDFToDouble(key) < 10.0))) (type: boolean) + Statistics: Num rows: 168 Data size: 1784 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string), '2008-04-08' (type: string), hr (type: string) outputColumnNames: _col0, _col1, _col2, _col3 - Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 168 Data size: 1784 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: select a.* from srcpart a where a.ds = '2008-04-08' and not(key > 50 or key < 10) and a.hr like '%2' diff --git ql/src/test/results/clientpositive/select_unquote_not.q.out ql/src/test/results/clientpositive/select_unquote_not.q.out index 0e59586..ff2c94f 100644 --- ql/src/test/results/clientpositive/select_unquote_not.q.out +++ ql/src/test/results/clientpositive/select_unquote_not.q.out @@ -59,15 +59,15 @@ STAGE PLANS: alias: npe_test Statistics: Num rows: 498 Data size: 5290 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (UDFToDouble(ds) >= 1970.0) (type: boolean) - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + predicate: (not (UDFToDouble(ds) < 1970.0)) (type: boolean) + Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string), ds (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat diff --git ql/src/test/results/clientpositive/semijoin4.q.out ql/src/test/results/clientpositive/semijoin4.q.out index 015dad1..9d92729 100644 --- ql/src/test/results/clientpositive/semijoin4.q.out +++ ql/src/test/results/clientpositive/semijoin4.q.out @@ -72,13 +72,13 @@ STAGE PLANS: predicate: ((UDFToInteger(tinyint_col_46) = -92) and decimal1309_col_65 is not null and bigint_col_13 is not null and UDFToInteger(tinyint_col_46) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: bigint_col_13 (type: bigint), smallint_col_24 (type: smallint), double_col_60 (type: double), decimal1309_col_65 (type: decimal(13,9)) - outputColumnNames: _col0, _col1, _col3, _col4 + expressions: bigint_col_13 (type: bigint), smallint_col_24 (type: smallint), tinyint_col_46 (type: tinyint), double_col_60 (type: double), decimal1309_col_65 (type: decimal(13,9)) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: bigint), _col4 (type: decimal(27,9)), -92 (type: tinyint) + key expressions: _col0 (type: bigint), _col4 (type: decimal(27,9)), _col2 (type: tinyint) sort order: +++ - Map-reduce partition columns: _col0 (type: bigint), _col4 (type: decimal(27,9)), -92 (type: tinyint) + Map-reduce partition columns: _col0 (type: bigint), _col4 (type: decimal(27,9)), _col2 (type: tinyint) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE value expressions: _col1 (type: smallint), _col3 (type: double) TableScan @@ -88,13 +88,13 @@ STAGE PLANS: predicate: ((UDFToInteger(tinyint_col_21) = -92) and tinyint_col_18 is not null and decimal2709_col_9 is not null and UDFToInteger(tinyint_col_21) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: decimal2709_col_9 (type: decimal(27,9)), tinyint_col_18 (type: tinyint) - outputColumnNames: _col0, _col1 + expressions: decimal2709_col_9 (type: decimal(27,9)), tinyint_col_18 (type: tinyint), tinyint_col_21 (type: tinyint) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: UDFToLong(_col1) (type: bigint), _col0 (type: decimal(27,9)), -92 (type: tinyint) + key expressions: UDFToLong(_col1) (type: bigint), _col0 (type: decimal(27,9)), _col2 (type: tinyint) sort order: +++ - Map-reduce partition columns: UDFToLong(_col1) (type: bigint), _col0 (type: decimal(27,9)), -92 (type: tinyint) + Map-reduce partition columns: UDFToLong(_col1) (type: bigint), _col0 (type: decimal(27,9)), _col2 (type: tinyint) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Operator Tree: Join Operator @@ -103,11 +103,11 @@ STAGE PLANS: keys: 0 _col0 (type: bigint), _col4 (type: decimal(27,9)), _col2 (type: tinyint) 1 UDFToLong(_col1) (type: bigint), _col0 (type: decimal(27,9)), _col2 (type: tinyint) - outputColumnNames: _col1, _col3 + outputColumnNames: _col1, _col3, _col7 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: smallint), _col3 (type: double) - outputColumnNames: _col0, _col1 + expressions: _col1 (type: smallint), _col3 (type: double), _col7 (type: tinyint), UDFToInteger(_col7) (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: false @@ -121,16 +121,16 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: -92 (type: int) + key expressions: _col3 (type: int) sort order: + - Map-reduce partition columns: -92 (type: int) + Map-reduce partition columns: _col3 (type: int) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col0 (type: smallint), _col1 (type: double) + value expressions: _col0 (type: smallint), _col1 (type: double), _col2 (type: tinyint) TableScan Reduce Output Operator - key expressions: -92 (type: int) + key expressions: _col0 (type: int) sort order: + - Map-reduce partition columns: -92 (type: int) + Map-reduce partition columns: _col0 (type: int) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Operator Tree: Join Operator @@ -139,7 +139,7 @@ STAGE PLANS: keys: 0 _col3 (type: int) 1 _col0 (type: int) - outputColumnNames: _col0, _col1 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator compressed: false @@ -153,14 +153,14 @@ STAGE PLANS: Map Operator Tree: TableScan Reduce Output Operator - key expressions: (-92 + _col0) (type: smallint), floor(_col1) (type: bigint) + key expressions: (UDFToShort(_col2) + _col0) (type: smallint), floor(_col1) (type: bigint) sort order: +- - Map-reduce partition columns: (-92 + _col0) (type: smallint) + Map-reduce partition columns: (UDFToShort(_col2) + _col0) (type: smallint) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col0 (type: smallint), _col1 (type: double) + value expressions: _col0 (type: smallint), _col1 (type: double), _col2 (type: tinyint) Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: double), -92 (type: tinyint) + expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: double), VALUE._col2 (type: tinyint) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE PTF Operator @@ -237,17 +237,21 @@ STAGE PLANS: 0 _col0 (type: decimal(19,11)), _col1 (type: timestamp) 1 _col0 (type: decimal(19,11)), _col1 (type: timestamp) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Group By Operator - keys: -92 (type: int) - mode: hash + Select Operator + expressions: -92 (type: int) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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 + Group By Operator + keys: _col0 (type: int) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL 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 Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/smb_mapjoin_25.q.out ql/src/test/results/clientpositive/smb_mapjoin_25.q.out index b0db59e..89b6993 100644 --- ql/src/test/results/clientpositive/smb_mapjoin_25.q.out +++ ql/src/test/results/clientpositive/smb_mapjoin_25.q.out @@ -46,8 +46,8 @@ POSTHOOK: query: load data local inpath '../../data/files/smbbucket_3.rc' overwr POSTHOOK: type: LOAD #### A masked pattern was here #### POSTHOOK: Output: default@smb_bucket_3 -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product -Warning: Shuffle Join JOIN[20][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-4:MAPRED' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[18][tables = [$hdt$_1, $hdt$_2]] in Stage 'Stage-4:MAPRED' is a cross product PREHOOK: query: explain select * from (select a.key from smb_bucket_1 a join smb_bucket_2 b on (a.key = b.key) where a.key = 5) t1 left outer join (select c.key from smb_bucket_2 c join smb_bucket_3 d on (c.key = d.key) where c.key=5) t2 on (t1.key=t2.key) where t2.key=5 PREHOOK: type: QUERY @@ -123,18 +123,15 @@ STAGE PLANS: keys: 0 5 (type: int) 1 5 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 5 (type: int), 5 (type: int) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 31 Data size: 129 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 + 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-4 Map Reduce @@ -182,8 +179,8 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Map Join MAPJOIN[37][bigTable=?] in task 'Stage-9:MAPRED' is a cross product -Warning: Map Join MAPJOIN[38][bigTable=?] in task 'Stage-10:MAPRED' is a cross product +Warning: Map Join MAPJOIN[35][bigTable=?] in task 'Stage-9:MAPRED' is a cross product +Warning: Map Join MAPJOIN[36][bigTable=?] in task 'Stage-10:MAPRED' is a cross product PREHOOK: query: -- explain -- select * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join src c on a.key=c.value @@ -289,18 +286,15 @@ STAGE PLANS: keys: 0 5 (type: int) 1 5 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 5 (type: int), 5 (type: int) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 31 Data size: 129 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 + 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 Local Work: Map Reduce Local Work @@ -328,18 +322,15 @@ STAGE PLANS: keys: 0 5 (type: int) 1 5 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 5 (type: int), 5 (type: int) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 31 Data size: 129 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 + 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 Local Work: Map Reduce Local Work @@ -365,18 +356,15 @@ STAGE PLANS: keys: 0 5 (type: int) 1 5 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 5 (type: int), 5 (type: int) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 31 Data size: 129 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 + 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-14 Map Reduce Local Work @@ -432,8 +420,8 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Map Join MAPJOIN[37][bigTable=?] in task 'Stage-9:MAPRED' is a cross product -Warning: Map Join MAPJOIN[38][bigTable=?] in task 'Stage-10:MAPRED' is a cross product +Warning: Map Join MAPJOIN[35][bigTable=?] in task 'Stage-9:MAPRED' is a cross product +Warning: Map Join MAPJOIN[36][bigTable=?] in task 'Stage-10:MAPRED' is a cross product PREHOOK: query: select * from (select a.key from smb_bucket_1 a join smb_bucket_2 b on (a.key = b.key) where a.key = 5) t1 left outer join (select c.key from smb_bucket_2 c join smb_bucket_3 d on (c.key = d.key) where c.key=5) t2 on (t1.key=t2.key) where t2.key=5 PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_1 diff --git ql/src/test/results/clientpositive/spark/auto_join8.q.out ql/src/test/results/clientpositive/spark/auto_join8.q.out index a769f4c..5427a0a 100644 --- ql/src/test/results/clientpositive/spark/auto_join8.q.out +++ ql/src/test/results/clientpositive/spark/auto_join8.q.out @@ -157,7 +157,7 @@ POSTHOOK: Input: default@src POSTHOOK: Output: default@dest1 POSTHOOK: Lineage: dest1.c1 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: dest1.c2 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] -POSTHOOK: Lineage: dest1.c3 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.c3 EXPRESSION [] POSTHOOK: Lineage: dest1.c4 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: SELECT sum(hash(dest1.c1,dest1.c2,dest1.c3,dest1.c4)) FROM dest1 PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/constprog_semijoin.q.out ql/src/test/results/clientpositive/spark/constprog_semijoin.q.out index 85387a7..0bb7362 100644 --- ql/src/test/results/clientpositive/spark/constprog_semijoin.q.out +++ ql/src/test/results/clientpositive/spark/constprog_semijoin.q.out @@ -455,7 +455,7 @@ STAGE PLANS: predicate: (((id = 100) = true) and (id <> 100) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: id (type: int), (id = 100) (type: boolean) + expressions: id (type: int), true (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -544,7 +544,7 @@ STAGE PLANS: predicate: ((id) IN (100, 200) and ((id = 100) = true) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: id (type: int), (id = 100) (type: boolean) + expressions: id (type: int), true (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -635,16 +635,18 @@ STAGE PLANS: predicate: (((id = 100) = true) and (id = 200) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: 200 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: 200 (type: int), false (type: boolean) + keys: _col0 (type: int), _col1 (type: boolean) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 200 (type: int), false (type: boolean) + key expressions: _col0 (type: int), _col1 (type: boolean) sort order: ++ - Map-reduce partition columns: 200 (type: int), false (type: boolean) + Map-reduce partition columns: _col0 (type: int), _col1 (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: @@ -652,7 +654,7 @@ STAGE PLANS: condition map: Left Semi Join 0 to 1 keys: - 0 _col3 (type: int), true (type: boolean) + 0 200 (type: int), true (type: boolean) 1 _col0 (type: int), _col1 (type: boolean) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2 Data size: 44 Basic stats: COMPLETE Column stats: NONE @@ -722,16 +724,18 @@ STAGE PLANS: predicate: (((id = 100) = true) and (id = 100) and (id = 100) is not null) (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: 100 (type: int), true (type: boolean) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: 100 (type: int), true (type: boolean) + keys: _col0 (type: int), _col1 (type: boolean) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 100 (type: int), true (type: boolean) + key expressions: _col0 (type: int), _col1 (type: boolean) sort order: ++ - Map-reduce partition columns: 100 (type: int), true (type: boolean) + Map-reduce partition columns: _col0 (type: int), _col1 (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: @@ -739,7 +743,7 @@ STAGE PLANS: condition map: Left Semi Join 0 to 1 keys: - 0 _col3 (type: int), true (type: boolean) + 0 100 (type: int), true (type: boolean) 1 _col0 (type: int), _col1 (type: boolean) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 2 Data size: 44 Basic stats: COMPLETE Column stats: NONE @@ -811,7 +815,7 @@ STAGE PLANS: predicate: (((id = 100) = true) and id is not null and (id = 100) is not null) (type: boolean) Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: id (type: int), (id = 100) (type: boolean) + expressions: id (type: int), true (type: boolean) outputColumnNames: _col0, _col1 Statistics: Num rows: 2 Data size: 6 Basic stats: COMPLETE Column stats: NONE Group By Operator diff --git ql/src/test/results/clientpositive/spark/dynamic_rdd_cache.q.out ql/src/test/results/clientpositive/spark/dynamic_rdd_cache.q.out index 8163773..333aa74 100644 --- ql/src/test/results/clientpositive/spark/dynamic_rdd_cache.q.out +++ ql/src/test/results/clientpositive/spark/dynamic_rdd_cache.q.out @@ -908,46 +908,46 @@ STAGE PLANS: outputColumnNames: _col2, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col6 (type: string), _col5 (type: int), _col4 (type: int), _col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col2 (type: int) + outputColumnNames: _col4, _col5, _col6, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator - aggregations: avg(_col4), stddev_samp(_col4) - keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), 4 (type: int) + aggregations: stddev_samp(_col2), avg(_col2) + keys: _col4 (type: int), _col5 (type: int), _col6 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), 4 (type: int) - sort order: ++++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int), 4 (type: int) + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: struct), _col5 (type: struct) + value expressions: _col3 (type: struct), _col4 (type: struct) Reducer 15 Reduce Operator Tree: Group By Operator - aggregations: avg(VALUE._col0), stddev_samp(VALUE._col1) - keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: int), 4 (type: int) + aggregations: stddev_samp(VALUE._col0), avg(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), _col5 (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col0 (type: int), _col3 (type: double), _col4 (type: double) + outputColumnNames: _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col5 / _col4) > 1.0)) END (type: boolean) + predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col3 / _col4) > 1.0)) END (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col5 / _col4)) END (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col2 (type: int), 4 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col3 / _col4)) END (type: double) + outputColumnNames: _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int), _col1 (type: int) sort order: ++ Map-reduce partition columns: _col2 (type: int), _col1 (type: int) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: double), _col5 (type: double) + value expressions: _col3 (type: int), _col4 (type: double), _col5 (type: double) Reducer 2 Reduce Operator Tree: Join Operator @@ -991,46 +991,46 @@ STAGE PLANS: outputColumnNames: _col2, _col4, _col5, _col6 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col6 (type: string), _col5 (type: int), _col4 (type: int), _col2 (type: int) - outputColumnNames: _col0, _col1, _col2, _col4 + expressions: _col4 (type: int), _col5 (type: int), _col6 (type: string), _col2 (type: int) + outputColumnNames: _col4, _col5, _col6, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Group By Operator - aggregations: avg(_col4), stddev_samp(_col4) - keys: _col0 (type: string), _col1 (type: int), _col2 (type: int), 3 (type: int) + aggregations: stddev_samp(_col2), avg(_col2) + keys: _col4 (type: int), _col5 (type: int), _col6 (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), _col1 (type: int), _col2 (type: int), 3 (type: int) - sort order: ++++ - Map-reduce partition columns: _col0 (type: string), _col1 (type: int), _col2 (type: int), 3 (type: int) + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: string) + sort order: +++ + Map-reduce partition columns: _col0 (type: int), _col1 (type: int), _col2 (type: string) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: struct), _col5 (type: struct) + value expressions: _col3 (type: struct), _col4 (type: struct) Reducer 5 Reduce Operator Tree: Group By Operator - aggregations: avg(VALUE._col0), stddev_samp(VALUE._col1) - keys: KEY._col0 (type: string), KEY._col1 (type: int), KEY._col2 (type: int), 3 (type: int) + aggregations: stddev_samp(VALUE._col0), avg(VALUE._col1) + keys: KEY._col0 (type: int), KEY._col1 (type: int), KEY._col2 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), _col5 (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col0 (type: int), _col3 (type: double), _col4 (type: double) + outputColumnNames: _col1, _col2, _col3, _col4 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Filter Operator - predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col5 / _col4) > 1.0)) END (type: boolean) + predicate: CASE WHEN ((_col4 = 0.0)) THEN (false) ELSE (((_col3 / _col4) > 1.0)) END (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col5 / _col4)) END (type: double) - outputColumnNames: _col1, _col2, _col4, _col5 + expressions: _col1 (type: int), _col2 (type: int), 3 (type: int), _col4 (type: double), CASE WHEN ((_col4 = 0.0)) THEN (null) ELSE ((_col3 / _col4)) END (type: double) + outputColumnNames: _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator key expressions: _col2 (type: int), _col1 (type: int) sort order: ++ Map-reduce partition columns: _col2 (type: int), _col1 (type: int) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - value expressions: _col4 (type: double), _col5 (type: double) + value expressions: _col3 (type: int), _col4 (type: double), _col5 (type: double) Reducer 6 Reduce Operator Tree: Join Operator @@ -1039,21 +1039,21 @@ STAGE PLANS: keys: 0 _col2 (type: int), _col1 (type: int) 1 _col2 (type: int), _col1 (type: int) - outputColumnNames: _col1, _col2, _col4, _col5, _col7, _col8, _col10, _col11 + outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col7, _col8, _col9, _col10, _col11 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: _col1 (type: int), _col2 (type: int), _col4 (type: double), _col5 (type: double), _col7 (type: int), _col8 (type: int), _col10 (type: double), _col11 (type: double) - outputColumnNames: _col0, _col1, _col3, _col4, _col5, _col6, _col8, _col9 + expressions: _col1 (type: int), _col2 (type: int), _col3 (type: int), _col4 (type: double), _col5 (type: double), _col7 (type: int), _col8 (type: int), _col9 (type: int), _col10 (type: double), _col11 (type: double) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: int), _col1 (type: int), 3 (type: int), _col3 (type: double), _col4 (type: double), 4 (type: int), _col8 (type: double), _col9 (type: double) + key expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: double), _col4 (type: double), _col7 (type: int), _col8 (type: double), _col9 (type: double) sort order: ++++++++ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE value expressions: _col5 (type: int), _col6 (type: int) Reducer 7 Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), 3 (type: int), KEY.reducesinkkey3 (type: double), KEY.reducesinkkey4 (type: double), VALUE._col0 (type: int), VALUE._col1 (type: int), 4 (type: int), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double) + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: int), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: double), KEY.reducesinkkey4 (type: double), VALUE._col0 (type: int), VALUE._col1 (type: int), KEY.reducesinkkey5 (type: int), KEY.reducesinkkey6 (type: double), KEY.reducesinkkey7 (type: double) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out index 1b4822e..e683fae 100644 --- ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out +++ ql/src/test/results/clientpositive/spark/groupby_sort_1_23.q.out @@ -816,16 +816,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string) - outputColumnNames: _col1 + outputColumnNames: key Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: 1 (type: int), _col1 (type: string) + aggregations: count(key) + keys: key (type: string) mode: final - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: 1 (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) + expressions: 1 (type: int), UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -944,7 +944,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl3 -POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl3.key1 SIMPLE [] POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] PREHOOK: query: SELECT * FROM outputTbl3 @@ -998,22 +998,22 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string) - null sort order: aaa - sort order: +++ - Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1073,12 +1073,12 @@ STAGE PLANS: Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 1 (type: int), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1144,7 +1144,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] @@ -3034,16 +3034,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -3172,7 +3172,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -3228,16 +3228,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string), 2 (type: int) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3, _col4 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), 2 (type: int), UDFToInteger(_col4) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), 2 (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -3356,7 +3356,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl5 -POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -3409,16 +3409,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -3607,16 +3607,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 2 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out index 0956771..435bb2c 100644 --- ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out +++ ql/src/test/results/clientpositive/spark/groupby_sort_skew_1_23.q.out @@ -835,16 +835,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string) - outputColumnNames: _col1 + outputColumnNames: key Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: 1 (type: int), _col1 (type: string) + aggregations: count(key) + keys: key (type: string) mode: final - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: 1 (type: int), UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int) + expressions: 1 (type: int), UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -963,7 +963,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl3 -POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl3.cnt EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl3.key1 SIMPLE [] POSTHOOK: Lineage: outputtbl3.key2 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] PREHOOK: query: SELECT * FROM outputTbl3 @@ -1018,22 +1018,22 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string) - null sort order: aaa - sort order: +++ + key expressions: _col0 (type: string), _col1 (type: string) + null sort order: aa + sort order: ++ Map-reduce partition columns: rand() (type: double) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) auto parallelism: false Path -> Alias: #### A masked pattern was here #### @@ -1093,30 +1093,30 @@ STAGE PLANS: Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 1 (type: int), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: partials - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string), 1 (type: int), _col2 (type: string) - null sort order: aaa - sort order: +++ - Map-reduce partition columns: _col0 (type: string), 1 (type: int), _col2 (type: string) + key expressions: _col0 (type: string), _col1 (type: string) + null sort order: aa + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE tag: -1 - value expressions: _col3 (type: bigint) + value expressions: _col2 (type: bigint) auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: KEY._col0 (type: string), 1 (type: int), KEY._col2 (type: string) + keys: KEY._col0 (type: string), KEY._col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1182,7 +1182,7 @@ SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t1 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t1)t1.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t1)t1.FieldSchema(name:val, type:string, comment:null), ] @@ -3167,16 +3167,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -3305,7 +3305,7 @@ SELECT key, 1, val, count(1) FROM T2 GROUP BY key, 1, val POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl4 -POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl4.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl4.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl4.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -3361,16 +3361,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: key, val Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string), 2 (type: int) + aggregations: count(val) + keys: key (type: string), val (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3, _col4 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), 2 (type: int), UDFToInteger(_col4) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), 2 (type: int), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -3489,7 +3489,7 @@ SELECT key, 1, val, 2, count(1) FROM T2 GROUP BY key, 1, val, 2 POSTHOOK: type: QUERY POSTHOOK: Input: default@t2 POSTHOOK: Output: default@outputtbl5 -POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.null, ] +POSTHOOK: Lineage: outputtbl5.cnt EXPRESSION [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key1 EXPRESSION [(t2)t2.FieldSchema(name:key, type:string, comment:null), ] POSTHOOK: Lineage: outputtbl5.key2 SIMPLE [] POSTHOOK: Lineage: outputtbl5.key3 SIMPLE [(t2)t2.FieldSchema(name:val, type:string, comment:null), ] @@ -3542,16 +3542,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 1 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 1 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -3740,16 +3740,16 @@ STAGE PLANS: GatherStats: false Select Operator expressions: key (type: string), val (type: string) - outputColumnNames: _col0, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count(1) - keys: _col0 (type: string), 2 (type: int), _col2 (type: string) + keys: _col0 (type: string), _col1 (type: string) mode: final - outputColumnNames: _col0, _col1, _col2, _col3 + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col2 (type: string), UDFToInteger(_col3) (type: int) + expressions: UDFToInteger(_col0) (type: int), 2 (type: int), _col1 (type: string), UDFToInteger(_col2) (type: int) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/spark/join28.q.out ql/src/test/results/clientpositive/spark/join28.q.out index 548a806..6f09ea3 100644 --- ql/src/test/results/clientpositive/spark/join28.q.out +++ ql/src/test/results/clientpositive/spark/join28.q.out @@ -47,7 +47,7 @@ STAGE PLANS: alias: z Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: key is not null (type: boolean) + predicate: ((11.0 = 11.0) and key is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) diff --git ql/src/test/results/clientpositive/spark/join32.q.out ql/src/test/results/clientpositive/spark/join32.q.out index fc3dcc2..5c26837 100644 --- ql/src/test/results/clientpositive/spark/join32.q.out +++ ql/src/test/results/clientpositive/spark/join32.q.out @@ -190,7 +190,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) diff --git ql/src/test/results/clientpositive/spark/join32_lessSize.q.out ql/src/test/results/clientpositive/spark/join32_lessSize.q.out index 4d5110f..833b394 100644 --- ql/src/test/results/clientpositive/spark/join32_lessSize.q.out +++ ql/src/test/results/clientpositive/spark/join32_lessSize.q.out @@ -198,7 +198,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -1008,7 +1008,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -1442,7 +1442,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -1892,7 +1892,7 @@ STAGE PLANS: alias: x Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) @@ -2136,7 +2136,7 @@ STAGE PLANS: alias: y Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) diff --git ql/src/test/results/clientpositive/spark/join33.q.out ql/src/test/results/clientpositive/spark/join33.q.out index fc3dcc2..5c26837 100644 --- ql/src/test/results/clientpositive/spark/join33.q.out +++ ql/src/test/results/clientpositive/spark/join33.q.out @@ -190,7 +190,7 @@ STAGE PLANS: GatherStats: false Filter Operator isSamplingPred: false - predicate: value is not null (type: boolean) + predicate: ((11.0 = 11.0) and value is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: value (type: string) diff --git ql/src/test/results/clientpositive/spark/join8.q.out ql/src/test/results/clientpositive/spark/join8.q.out index 270053c..412a9eb 100644 --- ql/src/test/results/clientpositive/spark/join8.q.out +++ ql/src/test/results/clientpositive/spark/join8.q.out @@ -161,7 +161,7 @@ POSTHOOK: Input: default@src POSTHOOK: Output: default@dest1 POSTHOOK: Lineage: dest1.c1 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] POSTHOOK: Lineage: dest1.c2 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] -POSTHOOK: Lineage: dest1.c3 EXPRESSION [(src)src1.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: dest1.c3 EXPRESSION [] POSTHOOK: Lineage: dest1.c4 SIMPLE [(src)src1.FieldSchema(name:value, type:string, comment:default), ] PREHOOK: query: SELECT dest1.* FROM dest1 PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out index 2d5d7c1..82f31e4 100644 --- ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out +++ ql/src/test/results/clientpositive/spark/join_filters_overlap.q.out @@ -112,7 +112,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -184,7 +184,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -343,7 +343,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -483,7 +483,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -642,7 +642,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -782,7 +782,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -1077,7 +1077,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -1149,7 +1149,7 @@ STAGE PLANS: predicate: (value = 40) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 40 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -1371,7 +1371,7 @@ STAGE PLANS: predicate: (value = 50) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 50 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -1443,7 +1443,7 @@ STAGE PLANS: predicate: (value = 60) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 60 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator @@ -1515,7 +1515,7 @@ STAGE PLANS: predicate: (value = 40) (type: boolean) Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: int), 40 (type: int) + expressions: key (type: int), value (type: int) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 6 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator diff --git ql/src/test/results/clientpositive/spark/join_view.q.out ql/src/test/results/clientpositive/spark/join_view.q.out index 14a28be..9d8f6d7 100644 --- ql/src/test/results/clientpositive/spark/join_view.q.out +++ ql/src/test/results/clientpositive/spark/join_view.q.out @@ -38,6 +38,7 @@ POSTHOOK: Input: default@invites POSTHOOK: Input: default@invites2 POSTHOOK: Output: database:default POSTHOOK: Output: default@v +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product PREHOOK: query: explain select * from v where ds='2011-09-01' PREHOOK: type: QUERY POSTHOOK: query: explain select * from v where ds='2011-09-01' @@ -50,7 +51,7 @@ STAGE PLANS: Stage: Stage-1 Spark Edges: - Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 3 (PARTITION-LEVEL SORT, 2) + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Map 3 (PARTITION-LEVEL SORT, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -66,9 +67,7 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: '2011-09-01' (type: string) - sort order: + - Map-reduce partition columns: '2011-09-01' (type: string) + sort order: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: string) Map 3 @@ -84,9 +83,7 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator - key expressions: '2011-09-01' (type: string) - sort order: + - Map-reduce partition columns: '2011-09-01' (type: string) + sort order: Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE value expressions: _col0 (type: int) Reducer 2 @@ -95,8 +92,8 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: string) - 1 _col1 (type: string) + 0 + 1 outputColumnNames: _col0, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator diff --git ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out index 84d99c3..5cda2d1 100644 --- ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out +++ ql/src/test/results/clientpositive/spark/load_dyn_part14.q.out @@ -63,8 +63,8 @@ STAGE PLANS: Spark Edges: Reducer 2 <- Map 1 (GROUP, 1) - Reducer 4 <- Map 1 (GROUP, 1) - Reducer 6 <- Map 1 (GROUP, 1) + Reducer 4 <- Map 3 (GROUP, 1) + Reducer 6 <- Map 3 (GROUP, 1) #### A masked pattern was here #### Vertices: Map 1 @@ -73,6 +73,8 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: null (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Limit Number of rows: 2 @@ -81,23 +83,42 @@ STAGE PLANS: sort order: Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE TopN Hash Memory Usage: 0.1 + value expressions: _col0 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE + Limit + Number of rows: 2 + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + Reduce Output Operator + sort order: + Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE + TopN Hash Memory Usage: 0.1 Reducer 2 Reduce Operator Tree: - Limit - Number of rows: 2 + Select Operator + expressions: VALUE._col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - expressions: 'k1' (type: string), null (type: string) - outputColumnNames: _col0, _col1 + Limit + Number of rows: 2 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 6 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - 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 - name: default.nzhang_part14 + Select Operator + expressions: 'k1' (type: string), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + File Output Operator + compressed: false + Statistics: Num rows: 6 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE + 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 + name: default.nzhang_part14 Reducer 4 Reduce Operator Tree: Limit diff --git ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out index 035bfc5..8ce21b6 100644 --- ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out +++ ql/src/test/results/clientpositive/spark/mapjoin_subquery.q.out @@ -34,7 +34,7 @@ STAGE PLANS: alias: z Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: key is not null (type: boolean) + predicate: ((11.0 = 11.0) and key is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) @@ -284,7 +284,7 @@ STAGE PLANS: alias: z Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: key is not null (type: boolean) + predicate: ((11.0 = 11.0) and key is not null) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string), value (type: string) diff --git ql/src/test/results/clientpositive/spark/pcr.q.out ql/src/test/results/clientpositive/spark/pcr.q.out index 3dd2147..5cd3f65 100644 --- ql/src/test/results/clientpositive/spark/pcr.q.out +++ ql/src/test/results/clientpositive/spark/pcr.q.out @@ -1547,7 +1547,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: 14 (type: int), KEY.reducesinkkey1 (type: string) + expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 20 Data size: 160 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -2445,19 +2445,23 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: a - sort order: + + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: int), _col4 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - tag: -1 - value expressions: _col1 (type: string), _col3 (type: int), _col4 (type: string) - auto parallelism: false + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: a + sort order: + + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string) + auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), '2000-04-08' (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col1 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -2710,19 +2714,23 @@ STAGE PLANS: 1 _col0 (type: int) outputColumnNames: _col0, _col1, _col3, _col4 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: int) - null sort order: a - sort order: + + Select Operator + expressions: _col0 (type: int), _col1 (type: string), '2000-04-08' (type: string), _col3 (type: int), _col4 (type: string), '2000-04-09' (type: string) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE - tag: -1 - value expressions: _col1 (type: string), _col3 (type: int), _col4 (type: string) - auto parallelism: false + Reduce Output Operator + key expressions: _col0 (type: int) + null sort order: a + sort order: + + Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE + tag: -1 + value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: int), _col4 (type: string), _col5 (type: string) + auto parallelism: false Reducer 3 Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), '2000-04-08' (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), '2000-04-09' (type: string) + expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: int), VALUE._col3 (type: string), VALUE._col4 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 22 Data size: 176 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4139,7 +4147,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), '2008-04-08' (type: string), KEY.reducesinkkey2 (type: string) + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -4324,7 +4332,7 @@ STAGE PLANS: Needs Tagging: false Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), '11' (type: string) + expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out index 312b3bd..ca6d1f8 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join4.q.out @@ -43,35 +43,35 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) + predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string) - outputColumnNames: _col0 + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Map 3 Map Operator Tree: TableScan alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) + predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) Map 4 Map Operator Tree: TableScan @@ -95,7 +95,7 @@ STAGE PLANS: Join Operator condition map: Inner Join 0 to 1 - Inner Join 1 to 2 + Inner Join 0 to 2 keys: 0 _col0 (type: string) 1 _col0 (type: string) @@ -103,7 +103,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col0 (type: string) + expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -417,35 +417,35 @@ STAGE PLANS: alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) + predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string) - outputColumnNames: _col0 + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Map 3 Map Operator Tree: TableScan alias: a Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((key > '10') and (key < '20') and (key > '15') and (key < '25') and (sqrt(key) <> 13.0)) (type: boolean) + predicate: ((sqrt(key) <> 13.0) and (key > '10') and (key < '20') and (key > '15') and (key < '25')) (type: boolean) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string) + outputColumnNames: _col0 Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 6 Data size: 63 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) Map 4 Map Operator Tree: TableScan @@ -469,7 +469,7 @@ STAGE PLANS: Join Operator condition map: Inner Join 0 to 1 - Inner Join 1 to 2 + Inner Join 0 to 2 keys: 0 _col0 (type: string) 1 _col0 (type: string) @@ -477,7 +477,7 @@ STAGE PLANS: outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col0 (type: string) + expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string), _col4 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4 Statistics: Num rows: 13 Data size: 138 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out index ef8c674..4b4c782 100644 --- ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out +++ ql/src/test/results/clientpositive/spark/ppd_outer_join5.q.out @@ -30,7 +30,7 @@ POSTHOOK: query: create table t4 (id int, key string, value string) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t4 -Warning: Shuffle Join JOIN[13][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product PREHOOK: query: explain select * from t1 full outer join t2 on t1.id=t2.id join t3 on t2.id=t3.id where t3.id=20 PREHOOK: type: QUERY POSTHOOK: query: explain select * from t1 full outer join t2 on t1.id=t2.id join t3 on t2.id=t3.id where t3.id=20 @@ -55,7 +55,7 @@ STAGE PLANS: predicate: (id = 20) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: 20 (type: int), key (type: string), value (type: string) + expressions: id (type: int), key (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator @@ -128,7 +128,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product PREHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t2.id=t3.id) where t2.id=20 PREHOOK: type: QUERY POSTHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t2.id=t3.id) where t2.id=20 @@ -185,7 +185,7 @@ STAGE PLANS: predicate: (id = 20) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: 20 (type: int), key (type: string), value (type: string) + expressions: id (type: int), key (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator @@ -222,7 +222,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[14][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[12][tables = [$hdt$_0, $hdt$_1, $hdt$_2]] in Work 'Reducer 2' is a cross product PREHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t1.id=t3.id) where t2.id=20 PREHOOK: type: QUERY POSTHOOK: query: explain select * from t1 join t2 on (t1.id=t2.id) left outer join t3 on (t1.id=t3.id) where t2.id=20 @@ -279,7 +279,7 @@ STAGE PLANS: predicate: (id = 20) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: 20 (type: int), key (type: string), value (type: string) + expressions: id (type: int), key (type: string), value (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Reduce Output Operator diff --git ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out index 4b392ba..28591e2 100644 --- ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out +++ ql/src/test/results/clientpositive/spark/smb_mapjoin_25.q.out @@ -46,8 +46,8 @@ POSTHOOK: query: load data local inpath '../../data/files/smbbucket_3.rc' overwr POSTHOOK: type: LOAD #### A masked pattern was here #### POSTHOOK: Output: default@smb_bucket_3 -Warning: Shuffle Join JOIN[9][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product -Warning: Shuffle Join JOIN[20][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 6' is a cross product +Warning: Shuffle Join JOIN[8][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product +Warning: Shuffle Join JOIN[18][tables = [$hdt$_1, $hdt$_2]] in Work 'Reducer 6' is a cross product PREHOOK: query: explain select * from (select a.key from smb_bucket_1 a join smb_bucket_2 b on (a.key = b.key) where a.key = 5) t1 left outer join (select c.key from smb_bucket_2 c join smb_bucket_3 d on (c.key = d.key) where c.key=5) t2 on (t1.key=t2.key) where t2.key=5 PREHOOK: type: QUERY @@ -141,18 +141,15 @@ STAGE PLANS: keys: 0 5 (type: int) 1 5 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 5 (type: int), 5 (type: int) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 31 Data size: 129 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 + 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 Reducer 6 Reduce Operator Tree: Join Operator @@ -174,8 +171,8 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Map Join MAPJOIN[33][bigTable=?] in task 'Stage-1:MAPRED' is a cross product -Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Stage-2:MAPRED' is a cross product +Warning: Map Join MAPJOIN[31][bigTable=?] in task 'Stage-1:MAPRED' is a cross product +Warning: Map Join MAPJOIN[30][bigTable=?] in task 'Stage-2:MAPRED' is a cross product PREHOOK: query: -- explain -- select * from smb_bucket_1 a left outer join smb_bucket_2 b on a.key = b.key left outer join src c on a.key=c.value @@ -295,20 +292,17 @@ STAGE PLANS: keys: 0 5 (type: int) 1 5 (type: int) + outputColumnNames: _col0, _col1 input vertices: 0 Map 1 Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 5 (type: int), 5 (type: int) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 31 Data size: 129 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 31 Data size: 129 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 + 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 Local Work: Map Reduce Local Work @@ -318,8 +312,8 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Map Join MAPJOIN[33][bigTable=?] in task 'Stage-1:MAPRED' is a cross product -Warning: Map Join MAPJOIN[32][bigTable=?] in task 'Stage-2:MAPRED' is a cross product +Warning: Map Join MAPJOIN[31][bigTable=?] in task 'Stage-1:MAPRED' is a cross product +Warning: Map Join MAPJOIN[30][bigTable=?] in task 'Stage-2:MAPRED' is a cross product PREHOOK: query: select * from (select a.key from smb_bucket_1 a join smb_bucket_2 b on (a.key = b.key) where a.key = 5) t1 left outer join (select c.key from smb_bucket_2 c join smb_bucket_3 d on (c.key = d.key) where c.key=5) t2 on (t1.key=t2.key) where t2.key=5 PREHOOK: type: QUERY PREHOOK: Input: default@smb_bucket_1 diff --git ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out index bbf24fd..8438176 100644 --- ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out +++ ql/src/test/results/clientpositive/spark/table_access_keys_stats.q.out @@ -77,7 +77,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key PREHOOK: type: QUERY PREHOOK: Input: default@t1 #### A masked pattern was here #### -Operator:GBY_3 +Operator:GBY_2 Table:default@t1 Keys:key @@ -90,7 +90,7 @@ PREHOOK: query: SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val PREHOOK: type: QUERY PREHOOK: Input: default@t1 #### A masked pattern was here #### -Operator:GBY_3 +Operator:GBY_2 Table:default@t1 Keys:key,val @@ -104,7 +104,7 @@ PREHOOK: query: SELECT key, 1, val, 2, count(1) FROM T1 GROUP BY key, 1, val, 2 PREHOOK: type: QUERY PREHOOK: Input: default@t1 #### A masked pattern was here #### -Operator:GBY_3 +Operator:GBY_2 Table:default@t1 Keys:key,val @@ -413,7 +413,7 @@ PREHOOK: Input: default@t2 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key @@ -439,7 +439,7 @@ Operator:JOIN_8 Table:default@t1 Keys:key Table:default@t2 -Keys:val,key +Keys:key PREHOOK: query: -- no mapping on functions SELECT * @@ -474,7 +474,7 @@ PREHOOK: Input: default@t2 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key @@ -505,7 +505,7 @@ PREHOOK: Input: default@t3 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key @@ -543,7 +543,7 @@ PREHOOK: Input: default@t3 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key diff --git ql/src/test/results/clientpositive/spark/union_remove_25.q.out ql/src/test/results/clientpositive/spark/union_remove_25.q.out index 253bf8f..d30a7bd 100644 --- ql/src/test/results/clientpositive/spark/union_remove_25.q.out +++ ql/src/test/results/clientpositive/spark/union_remove_25.q.out @@ -268,8 +268,8 @@ STAGE PLANS: alias: srcpart Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 + expressions: key (type: string), value (type: string), '2008-04-08' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 500 @@ -278,18 +278,18 @@ STAGE PLANS: sort order: Statistics: Num rows: 500 Data size: 5000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - value expressions: _col0 (type: string), _col1 (type: string) + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) Reducer 2 Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) - outputColumnNames: _col0, _col1 + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5000 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 500 Statistics: Num rows: 500 Data size: 5000 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string) + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -303,14 +303,14 @@ STAGE PLANS: Reducer 4 Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), VALUE._col1 (type: string) - outputColumnNames: _col0, _col1 + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5000 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 500 Statistics: Num rows: 500 Data size: 5000 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string) + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -437,8 +437,8 @@ STAGE PLANS: alias: srcpart Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: key (type: string), value (type: string), hr (type: string) - outputColumnNames: _col0, _col1, _col3 + expressions: key (type: string), value (type: string), '2008-04-08' (type: string), hr (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 1000 @@ -447,18 +447,18 @@ STAGE PLANS: sort order: Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - value expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string) + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) Reducer 2 Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col3 (type: string) - outputColumnNames: _col0, _col1, _col3 + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 1000 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string) + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string), _col3 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -472,14 +472,14 @@ STAGE PLANS: Reducer 4 Reduce Operator Tree: Select Operator - expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col3 (type: string) - outputColumnNames: _col0, _col1, _col3 + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE Limit Number of rows: 1000 Statistics: Num rows: 1000 Data size: 10000 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), '2008-04-08' (type: string), _col3 (type: string) + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string), _col3 (type: string) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 2000 Data size: 20000 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/spark/union_view.q.out ql/src/test/results/clientpositive/spark/union_view.q.out index 402d9fd..2a97531 100644 --- ql/src/test/results/clientpositive/spark/union_view.q.out +++ ql/src/test/results/clientpositive/spark/union_view.q.out @@ -271,20 +271,16 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 Map 2 Map Operator Tree: TableScan @@ -295,20 +291,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '1')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 Map 3 Map Operator Tree: TableScan @@ -319,20 +311,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '1')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 @@ -359,20 +347,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '2')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '2' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 Map 2 Map Operator Tree: TableScan @@ -383,20 +367,16 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '2' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 Map 3 Map Operator Tree: TableScan @@ -407,20 +387,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '2')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '2' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 @@ -447,20 +423,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '3')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '3' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 Map 2 Map Operator Tree: TableScan @@ -471,20 +443,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '3')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '3' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 Map 3 Map Operator Tree: TableScan @@ -495,20 +463,16 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '3' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 @@ -537,18 +501,14 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), ds (type: string) - outputColumnNames: _col1, _col2 + expressions: 86 (type: int), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col1 (type: string), _col2 (type: string) - outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col2 (type: string) - sort order: + - Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Map 3 Map Operator Tree: TableScan @@ -559,18 +519,14 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), ds (type: string) - outputColumnNames: _col1, _col2 + expressions: 86 (type: int), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col1 (type: string), _col2 (type: string) - outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col2 (type: string) - sort order: + - Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Map 4 Map Operator Tree: TableScan @@ -581,22 +537,18 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), ds (type: string) - outputColumnNames: _col1, _col2 + expressions: 86 (type: int), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col1 (type: string), _col2 (type: string) - outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col2 (type: string) - sort order: + - Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Reducer 2 Reduce Operator Tree: Select Operator - expressions: 86 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string) + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -930,20 +882,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '4')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '4' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 Map 2 Map Operator Tree: TableScan @@ -954,20 +902,16 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '4')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '4' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 Map 3 Map Operator Tree: TableScan @@ -978,20 +922,16 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '4' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 diff --git ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out index 8b3d353..4ace18b 100644 --- ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/spark/vector_mapjoin_reduce.q.out @@ -205,18 +205,18 @@ STAGE PLANS: predicate: ((l_shipmode = 'AIR') and (l_linenumber = 1) and l_orderkey is not null) (type: boolean) Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: l_orderkey (type: int) - outputColumnNames: _col0 + expressions: l_orderkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: _col0 (type: int), 1 (type: int) + keys: _col0 (type: int), _col1 (type: int) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Spark HashTable Sink Operator keys: - 0 _col0 (type: int), 1 (type: int) - 1 _col0 (type: int), 1 (type: int) + 0 _col0 (type: int), _col3 (type: int) + 1 _col0 (type: int), _col1 (type: int) Local Work: Map Reduce Local Work Map 3 @@ -265,15 +265,15 @@ STAGE PLANS: predicate: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean) Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int) - outputColumnNames: _col0, _col1, _col2 + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), l_linenumber (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Left Semi Join 0 to 1 keys: - 0 _col0 (type: int), 1 (type: int) - 1 _col0 (type: int), 1 (type: int) + 0 _col0 (type: int), _col3 (type: int) + 1 _col0 (type: int), _col1 (type: int) outputColumnNames: _col1, _col2 input vertices: 1 Map 2 diff --git ql/src/test/results/clientpositive/subquery_notin.q.out ql/src/test/results/clientpositive/subquery_notin.q.out index c600b7f..95ae1f4 100644 --- ql/src/test/results/clientpositive/subquery_notin.q.out +++ ql/src/test/results/clientpositive/subquery_notin.q.out @@ -1,4 +1,4 @@ -Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: -- non agg, non corr explain select * @@ -29,32 +29,32 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((key > '2') and key is null) (type: boolean) - Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE - Select Operator - Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: false (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (_col0 = 0) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false table: @@ -68,18 +68,21 @@ STAGE PLANS: TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: key (type: string), value (type: string) - outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: string), _col1 (type: string) + Filter Operator + predicate: false (type: boolean) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) TableScan Reduce Output Operator sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Operator Tree: Join Operator condition map: @@ -88,7 +91,7 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false table: @@ -104,23 +107,23 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: string) TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (key > '2') (type: boolean) - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + predicate: false (type: boolean) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Join Operator condition map: @@ -129,17 +132,17 @@ STAGE PLANS: 0 _col0 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: _col3 is null (type: boolean) - Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: string) outputColumnNames: _col0, _col1 - Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 302 Data size: 3208 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -151,7 +154,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select * from src where src.key not in ( select key from src s1 where s1.key > '2') @@ -166,126 +169,7 @@ order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src #### A masked pattern was here #### -0 val_0 -0 val_0 -0 val_0 -10 val_10 -100 val_100 -100 val_100 -103 val_103 -103 val_103 -104 val_104 -104 val_104 -105 val_105 -11 val_11 -111 val_111 -113 val_113 -113 val_113 -114 val_114 -116 val_116 -118 val_118 -118 val_118 -119 val_119 -119 val_119 -119 val_119 -12 val_12 -12 val_12 -120 val_120 -120 val_120 -125 val_125 -125 val_125 -126 val_126 -128 val_128 -128 val_128 -128 val_128 -129 val_129 -129 val_129 -131 val_131 -133 val_133 -134 val_134 -134 val_134 -136 val_136 -137 val_137 -137 val_137 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -143 val_143 -145 val_145 -146 val_146 -146 val_146 -149 val_149 -149 val_149 -15 val_15 -15 val_15 -150 val_150 -152 val_152 -152 val_152 -153 val_153 -155 val_155 -156 val_156 -157 val_157 -158 val_158 -160 val_160 -162 val_162 -163 val_163 -164 val_164 -164 val_164 -165 val_165 -165 val_165 -166 val_166 -167 val_167 -167 val_167 -167 val_167 -168 val_168 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -17 val_17 -170 val_170 -172 val_172 -172 val_172 -174 val_174 -174 val_174 -175 val_175 -175 val_175 -176 val_176 -176 val_176 -177 val_177 -178 val_178 -179 val_179 -179 val_179 -18 val_18 -18 val_18 -180 val_180 -181 val_181 -183 val_183 -186 val_186 -187 val_187 -187 val_187 -187 val_187 -189 val_189 -19 val_19 -190 val_190 -191 val_191 -191 val_191 -192 val_192 -193 val_193 -193 val_193 -193 val_193 -194 val_194 -195 val_195 -195 val_195 -196 val_196 -197 val_197 -197 val_197 -199 val_199 -199 val_199 -199 val_199 -2 val_2 -Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: -- non agg, corr explain select p_mfgr, b.p_name, p_size @@ -530,7 +414,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[27][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[26][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select p_mfgr, b.p_name, p_size from part b where b.p_name not in @@ -569,7 +453,7 @@ Manufacturer#4 almond azure aquamarine papaya violet 12 Manufacturer#5 almond antique blue firebrick mint 31 Manufacturer#5 almond aquamarine dodger light gainsboro 46 Manufacturer#5 almond azure blanched chiffon midnight 23 -Warning: Shuffle Join JOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: -- agg, non corr explain select p_name, p_size @@ -851,7 +735,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[38][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select p_name, p_size from part where part.p_size not in @@ -898,7 +782,7 @@ almond aquamarine sandy cyan gainsboro 18 almond aquamarine yellow dodger mint 7 almond azure aquamarine papaya violet 12 almond azure blanched chiffon midnight 23 -Warning: Shuffle Join JOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: -- agg, corr explain select p_mfgr, p_name, p_size @@ -1212,7 +1096,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[37][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[36][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select p_mfgr, p_name, p_size from part b where b.p_size not in (select min(p_size) @@ -1253,7 +1137,7 @@ Manufacturer#5 almond antique medium spring khaki 6 Manufacturer#5 almond azure blanched chiffon midnight 23 Manufacturer#5 almond antique blue firebrick mint 31 Manufacturer#5 almond aquamarine dodger light gainsboro 46 -Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: -- non agg, non corr, Group By in Parent Query select li.l_partkey, count(*) from lineitem li @@ -1288,7 +1172,7 @@ POSTHOOK: Input: default@lineitem 139636 1 175839 1 182052 1 -Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: -- alternate not in syntax select * from src @@ -1305,125 +1189,6 @@ order by key POSTHOOK: type: QUERY POSTHOOK: Input: default@src #### A masked pattern was here #### -0 val_0 -0 val_0 -0 val_0 -10 val_10 -100 val_100 -100 val_100 -103 val_103 -103 val_103 -104 val_104 -104 val_104 -105 val_105 -11 val_11 -111 val_111 -113 val_113 -113 val_113 -114 val_114 -116 val_116 -118 val_118 -118 val_118 -119 val_119 -119 val_119 -119 val_119 -12 val_12 -12 val_12 -120 val_120 -120 val_120 -125 val_125 -125 val_125 -126 val_126 -128 val_128 -128 val_128 -128 val_128 -129 val_129 -129 val_129 -131 val_131 -133 val_133 -134 val_134 -134 val_134 -136 val_136 -137 val_137 -137 val_137 -138 val_138 -138 val_138 -138 val_138 -138 val_138 -143 val_143 -145 val_145 -146 val_146 -146 val_146 -149 val_149 -149 val_149 -15 val_15 -15 val_15 -150 val_150 -152 val_152 -152 val_152 -153 val_153 -155 val_155 -156 val_156 -157 val_157 -158 val_158 -160 val_160 -162 val_162 -163 val_163 -164 val_164 -164 val_164 -165 val_165 -165 val_165 -166 val_166 -167 val_167 -167 val_167 -167 val_167 -168 val_168 -169 val_169 -169 val_169 -169 val_169 -169 val_169 -17 val_17 -170 val_170 -172 val_172 -172 val_172 -174 val_174 -174 val_174 -175 val_175 -175 val_175 -176 val_176 -176 val_176 -177 val_177 -178 val_178 -179 val_179 -179 val_179 -18 val_18 -18 val_18 -180 val_180 -181 val_181 -183 val_183 -186 val_186 -187 val_187 -187 val_187 -187 val_187 -189 val_189 -19 val_19 -190 val_190 -191 val_191 -191 val_191 -192 val_192 -193 val_193 -193 val_193 -193 val_193 -194 val_194 -195 val_195 -195 val_195 -196 val_196 -197 val_197 -197 val_197 -199 val_199 -199 val_199 -199 val_199 -2 val_2 PREHOOK: query: -- null check create view T1_v as select key from src where key <'11' @@ -1452,7 +1217,7 @@ POSTHOOK: Input: default@src POSTHOOK: Input: default@t1_v POSTHOOK: Output: database:default POSTHOOK: Output: default@T2_v -Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: explain select * from T1_v where T1_v.key not in (select T2_v.key from T2_v) @@ -1556,17 +1321,17 @@ STAGE PLANS: alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: ((key < '11') and (CASE WHEN ((key > '104')) THEN (null) ELSE (key) END < '11')) (type: boolean) - Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE + predicate: ((key < '11') and CASE WHEN ((key > '104')) THEN (null) ELSE ((key < '11')) END) (type: boolean) + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: CASE WHEN ((key > '104')) THEN (null) ELSE (key) END (type: string) outputColumnNames: _col0 - Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 55 Data size: 584 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Join Operator condition map: @@ -1597,7 +1362,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[18][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product +Warning: Shuffle Join JOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product PREHOOK: query: select * from T1_v where T1_v.key not in (select T2_v.key from T2_v) PREHOOK: type: QUERY diff --git ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out index 70f9591..d6d3319 100644 --- ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out +++ ql/src/test/results/clientpositive/subquery_notin_having.q.java1.7.out @@ -1,4 +1,4 @@ -Warning: Shuffle Join JOIN[21][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[22][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product PREHOOK: query: -- non agg, non corr -- JAVA_VERSION_SPECIFIC_OUTPUT @@ -37,21 +37,20 @@ STAGE PLANS: TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: key (type: string) - outputColumnNames: key - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: false (type: boolean) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() keys: key (type: string) mode: hash outputColumnNames: _col0, _col1 - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) Reduce Operator Tree: Group By Operator @@ -59,7 +58,7 @@ STAGE PLANS: keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0, _col1 - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false table: @@ -73,12 +72,12 @@ STAGE PLANS: TableScan Reduce Output Operator sort order: - Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: string), _col1 (type: bigint) TableScan Reduce Output Operator sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Operator Tree: Join Operator condition map: @@ -87,7 +86,7 @@ STAGE PLANS: 0 1 outputColumnNames: _col0, _col1 - Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false table: @@ -103,23 +102,23 @@ STAGE PLANS: key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 11 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: bigint) TableScan alias: src Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Filter Operator - predicate: (key > '12') (type: boolean) - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + predicate: false (type: boolean) + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: key (type: string) outputColumnNames: _col0 - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: string) sort order: + Map-reduce partition columns: _col0 (type: string) - Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 10 Basic stats: COMPLETE Column stats: NONE Reduce Operator Tree: Join Operator condition map: @@ -128,17 +127,17 @@ STAGE PLANS: 0 _col0 (type: string) 1 _col0 (type: string) outputColumnNames: _col0, _col1, _col3 - Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: _col3 is null (type: boolean) - Statistics: Num rows: 151 Data size: 1606 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: _col0 (type: string), _col1 (type: bigint) outputColumnNames: _col0, _col1 - Statistics: Num rows: 151 Data size: 1606 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 151 Data size: 1606 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat @@ -149,32 +148,32 @@ STAGE PLANS: Map Operator Tree: TableScan alias: src - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Filter Operator - predicate: ((key > '12') and key is null) (type: boolean) - Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE - Select Operator - Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE + Select Operator + Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE + Filter Operator + predicate: false (type: boolean) + Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE Group By Operator aggregations: count() mode: hash outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator sort order: - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE value expressions: _col0 (type: bigint) Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) mode: mergepartial outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Filter Operator predicate: (_col0 = 0) (type: boolean) - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator compressed: false table: @@ -188,7 +187,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product PREHOOK: query: -- non agg, corr explain select b.p_mfgr, min(p_retailprice) @@ -445,7 +444,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join JOIN[30][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product +Warning: Shuffle Join JOIN[29][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-2:MAPRED' is a cross product PREHOOK: query: select b.p_mfgr, min(p_retailprice) from part b group by b.p_mfgr @@ -619,25 +618,25 @@ STAGE PLANS: predicate: p_mfgr is null (type: boolean) Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: null (type: string), p_retailprice (type: double) - outputColumnNames: p_mfgr, p_retailprice + expressions: p_retailprice (type: double) + outputColumnNames: _col1 Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: max(p_retailprice), min(p_retailprice) - keys: p_mfgr (type: string) + aggregations: max(_col1), min(_col1) + keys: null (type: void) mode: hash outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: string) + key expressions: _col0 (type: void) sort order: + - Map-reduce partition columns: _col0 (type: string) + Map-reduce partition columns: _col0 (type: void) Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE value expressions: _col1 (type: double), _col2 (type: double) Reduce Operator Tree: Group By Operator aggregations: max(VALUE._col0), min(VALUE._col1) - keys: KEY._col0 (type: string) + keys: KEY._col0 (type: void) mode: mergepartial outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6 Data size: 726 Basic stats: COMPLETE Column stats: NONE diff --git ql/src/test/results/clientpositive/table_access_keys_stats.q.out ql/src/test/results/clientpositive/table_access_keys_stats.q.out index bbf24fd..8438176 100644 --- ql/src/test/results/clientpositive/table_access_keys_stats.q.out +++ ql/src/test/results/clientpositive/table_access_keys_stats.q.out @@ -77,7 +77,7 @@ SELECT 1, key, count(1) FROM T1 GROUP BY 1, key PREHOOK: type: QUERY PREHOOK: Input: default@t1 #### A masked pattern was here #### -Operator:GBY_3 +Operator:GBY_2 Table:default@t1 Keys:key @@ -90,7 +90,7 @@ PREHOOK: query: SELECT key, 1, val, count(1) FROM T1 GROUP BY key, 1, val PREHOOK: type: QUERY PREHOOK: Input: default@t1 #### A masked pattern was here #### -Operator:GBY_3 +Operator:GBY_2 Table:default@t1 Keys:key,val @@ -104,7 +104,7 @@ PREHOOK: query: SELECT key, 1, val, 2, count(1) FROM T1 GROUP BY key, 1, val, 2 PREHOOK: type: QUERY PREHOOK: Input: default@t1 #### A masked pattern was here #### -Operator:GBY_3 +Operator:GBY_2 Table:default@t1 Keys:key,val @@ -413,7 +413,7 @@ PREHOOK: Input: default@t2 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key @@ -439,7 +439,7 @@ Operator:JOIN_8 Table:default@t1 Keys:key Table:default@t2 -Keys:val,key +Keys:key PREHOOK: query: -- no mapping on functions SELECT * @@ -474,7 +474,7 @@ PREHOOK: Input: default@t2 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key @@ -505,7 +505,7 @@ PREHOOK: Input: default@t3 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key @@ -543,7 +543,7 @@ PREHOOK: Input: default@t3 #### A masked pattern was here #### Operator:JOIN_8 Table:default@t1 -Keys:val,key +Keys:key Table:default@t2 Keys:key diff --git ql/src/test/results/clientpositive/tez/constprog_semijoin.q.out ql/src/test/results/clientpositive/tez/constprog_semijoin.q.out index 636410a..0a8aea8 100644 --- ql/src/test/results/clientpositive/tez/constprog_semijoin.q.out +++ ql/src/test/results/clientpositive/tez/constprog_semijoin.q.out @@ -261,7 +261,7 @@ Stage-0 Stage-1 Reducer 2 File Output Operator [FS_12] - Merge Join Operator [MERGEJOIN_18] (rows=5 width=22) + Merge Join Operator [MERGEJOIN_17] (rows=5 width=22) Conds:RS_8._col3, true=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1","_col2"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_8] @@ -279,7 +279,7 @@ Stage-0 Output:["_col0","_col1"],keys:_col0, _col1 Select Operator [SEL_5] (rows=2 width=3) Output:["_col0","_col1"] - Filter Operator [FIL_17] (rows=2 width=3) + Filter Operator [FIL_16] (rows=2 width=3) predicate:(((id = 100) = true) and (id <> 100) and (id = 100) is not null) TableScan [TS_3] (rows=5 width=3) default@table3,table3,Tbl:COMPLETE,Col:NONE,Output:["id"] @@ -309,7 +309,7 @@ Stage-0 Stage-1 Reducer 2 File Output Operator [FS_12] - Merge Join Operator [MERGEJOIN_18] (rows=2 width=22) + Merge Join Operator [MERGEJOIN_17] (rows=2 width=22) Conds:RS_8._col3, true=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1","_col2"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_8] @@ -327,7 +327,7 @@ Stage-0 Output:["_col0","_col1"],keys:_col0, _col1 Select Operator [SEL_5] (rows=1 width=3) Output:["_col0","_col1"] - Filter Operator [FIL_17] (rows=1 width=3) + Filter Operator [FIL_16] (rows=1 width=3) predicate:((id) IN (100, 200) and ((id = 100) = true) and (id = 100) is not null) TableScan [TS_3] (rows=5 width=3) default@table3,table3,Tbl:COMPLETE,Col:NONE,Output:["id"] @@ -359,8 +359,8 @@ Stage-0 Stage-1 Reducer 2 File Output Operator [FS_12] - Merge Join Operator [MERGEJOIN_18] (rows=2 width=22) - Conds:RS_8.200, true=RS_9.200, false(Left Semi),Output:["_col0","_col1","_col2"] + Merge Join Operator [MERGEJOIN_17] (rows=2 width=22) + Conds:RS_8.200, true=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1","_col2"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_8] PartitionCols:200, true @@ -372,11 +372,12 @@ Stage-0 default@table1,table1,Tbl:COMPLETE,Col:NONE,Output:["id","val","val1","dimid"] <-Map 3 [SIMPLE_EDGE] SHUFFLE [RS_9] - PartitionCols:200, false + PartitionCols:_col0, _col1 Group By Operator [GBY_7] (rows=1 width=3) - Output:["_col0","_col1"],keys:200, false + Output:["_col0","_col1"],keys:_col0, _col1 Select Operator [SEL_5] (rows=1 width=3) - Filter Operator [FIL_17] (rows=1 width=3) + Output:["_col0","_col1"] + Filter Operator [FIL_16] (rows=1 width=3) predicate:(((id = 100) = true) and (id = 200) and (id = 100) is not null) TableScan [TS_3] (rows=5 width=3) default@table3,table3,Tbl:COMPLETE,Col:NONE,Output:["id"] @@ -406,8 +407,8 @@ Stage-0 Stage-1 Reducer 2 File Output Operator [FS_12] - Merge Join Operator [MERGEJOIN_18] (rows=2 width=22) - Conds:RS_8.100, true=RS_9.100, true(Left Semi),Output:["_col0","_col1","_col2"] + Merge Join Operator [MERGEJOIN_17] (rows=2 width=22) + Conds:RS_8.100, true=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1","_col2"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_8] PartitionCols:100, true @@ -419,11 +420,12 @@ Stage-0 default@table1,table1,Tbl:COMPLETE,Col:NONE,Output:["id","val","val1","dimid"] <-Map 3 [SIMPLE_EDGE] SHUFFLE [RS_9] - PartitionCols:100, true + PartitionCols:_col0, _col1 Group By Operator [GBY_7] (rows=1 width=3) - Output:["_col0","_col1"],keys:100, true + Output:["_col0","_col1"],keys:_col0, _col1 Select Operator [SEL_5] (rows=1 width=3) - Filter Operator [FIL_17] (rows=1 width=3) + Output:["_col0","_col1"] + Filter Operator [FIL_16] (rows=1 width=3) predicate:(((id = 100) = true) and (id = 100) and (id = 100) is not null) TableScan [TS_3] (rows=5 width=3) default@table3,table3,Tbl:COMPLETE,Col:NONE,Output:["id"] @@ -455,7 +457,7 @@ Stage-0 Stage-1 Reducer 2 File Output Operator [FS_12] - Merge Join Operator [MERGEJOIN_18] (rows=5 width=22) + Merge Join Operator [MERGEJOIN_17] (rows=5 width=22) Conds:RS_8._col3, true=RS_9._col0, _col1(Left Semi),Output:["_col0","_col1","_col2"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_8] @@ -473,7 +475,7 @@ Stage-0 Output:["_col0","_col1"],keys:_col0, _col1 Select Operator [SEL_5] (rows=2 width=3) Output:["_col0","_col1"] - Filter Operator [FIL_17] (rows=2 width=3) + Filter Operator [FIL_16] (rows=2 width=3) predicate:(((id = 100) = true) and id is not null and (id = 100) is not null) TableScan [TS_3] (rows=5 width=3) default@table3,table3,Tbl:COMPLETE,Col:NONE,Output:["id"] diff --git ql/src/test/results/clientpositive/tez/cte_mat_1.q.out ql/src/test/results/clientpositive/tez/cte_mat_1.q.out index bbe4296..7464a07 100644 --- ql/src/test/results/clientpositive/tez/cte_mat_1.q.out +++ ql/src/test/results/clientpositive/tez/cte_mat_1.q.out @@ -21,24 +21,22 @@ Stage-0 Stage-1 Reducer 2 File Output Operator [FS_10] - Select Operator [SEL_9] (rows=275 width=10) - Output:["_col0"] - Merge Join Operator [MERGEJOIN_15] (rows=275 width=10) - Conds:RS_6.'5'=RS_7.'5'(Inner) - <-Map 1 [SIMPLE_EDGE] - SHUFFLE [RS_6] - PartitionCols:'5' - Select Operator [SEL_2] (rows=250 width=10) - Filter Operator [FIL_13] (rows=250 width=10) - predicate:(key = '5') - TableScan [TS_0] (rows=500 width=10) - default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] - <-Map 3 [SIMPLE_EDGE] - SHUFFLE [RS_7] - PartitionCols:'5' - Select Operator [SEL_5] (rows=250 width=10) - Filter Operator [FIL_14] (rows=250 width=10) - predicate:(key = '5') - TableScan [TS_3] (rows=500 width=10) - default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] + Merge Join Operator [MERGEJOIN_15] (rows=275 width=10) + Conds:RS_6.'5'=RS_7.'5'(Inner),Output:["_col0"] + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_6] + PartitionCols:'5' + Select Operator [SEL_2] (rows=250 width=10) + Filter Operator [FIL_13] (rows=250 width=10) + predicate:(key = '5') + TableScan [TS_0] (rows=500 width=10) + default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] + <-Map 3 [SIMPLE_EDGE] + SHUFFLE [RS_7] + PartitionCols:'5' + Select Operator [SEL_5] (rows=250 width=10) + Filter Operator [FIL_14] (rows=250 width=10) + predicate:(key = '5') + TableScan [TS_3] (rows=500 width=10) + default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] diff --git ql/src/test/results/clientpositive/tez/cte_mat_2.q.out ql/src/test/results/clientpositive/tez/cte_mat_2.q.out index bbe4296..7464a07 100644 --- ql/src/test/results/clientpositive/tez/cte_mat_2.q.out +++ ql/src/test/results/clientpositive/tez/cte_mat_2.q.out @@ -21,24 +21,22 @@ Stage-0 Stage-1 Reducer 2 File Output Operator [FS_10] - Select Operator [SEL_9] (rows=275 width=10) - Output:["_col0"] - Merge Join Operator [MERGEJOIN_15] (rows=275 width=10) - Conds:RS_6.'5'=RS_7.'5'(Inner) - <-Map 1 [SIMPLE_EDGE] - SHUFFLE [RS_6] - PartitionCols:'5' - Select Operator [SEL_2] (rows=250 width=10) - Filter Operator [FIL_13] (rows=250 width=10) - predicate:(key = '5') - TableScan [TS_0] (rows=500 width=10) - default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] - <-Map 3 [SIMPLE_EDGE] - SHUFFLE [RS_7] - PartitionCols:'5' - Select Operator [SEL_5] (rows=250 width=10) - Filter Operator [FIL_14] (rows=250 width=10) - predicate:(key = '5') - TableScan [TS_3] (rows=500 width=10) - default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] + Merge Join Operator [MERGEJOIN_15] (rows=275 width=10) + Conds:RS_6.'5'=RS_7.'5'(Inner),Output:["_col0"] + <-Map 1 [SIMPLE_EDGE] + SHUFFLE [RS_6] + PartitionCols:'5' + Select Operator [SEL_2] (rows=250 width=10) + Filter Operator [FIL_13] (rows=250 width=10) + predicate:(key = '5') + TableScan [TS_0] (rows=500 width=10) + default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] + <-Map 3 [SIMPLE_EDGE] + SHUFFLE [RS_7] + PartitionCols:'5' + Select Operator [SEL_5] (rows=250 width=10) + Filter Operator [FIL_14] (rows=250 width=10) + predicate:(key = '5') + TableScan [TS_3] (rows=500 width=10) + default@src,src,Tbl:COMPLETE,Col:NONE,Output:["key"] diff --git ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out index 2626768..31bb0f1 100644 --- ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out +++ ql/src/test/results/clientpositive/tez/dynamic_partition_pruning.q.out @@ -1562,12 +1562,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1588,7 +1588,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1667,9 +1667,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: @@ -1710,7 +1710,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1808,9 +1808,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: @@ -1819,7 +1819,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1898,9 +1898,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: @@ -1926,7 +1926,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -2019,9 +2019,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + key expressions: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) sort order: + - Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: @@ -2062,7 +2062,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + 0 UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) 1 UDFToString(_col0) (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -2174,9 +2174,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: @@ -2185,7 +2185,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -2213,32 +2213,30 @@ STAGE PLANS: Reducer 5 Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -2276,7 +2274,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 #### A masked pattern was here #### 1000 -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: -- non-equi join EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY @@ -2371,7 +2369,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY PREHOOK: Input: default@srcpart @@ -2870,17 +2868,21 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Map 5 Map Operator Tree: TableScan @@ -4534,7 +4536,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) input vertices: 1 Map 3 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -4562,12 +4564,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -4652,7 +4654,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) input vertices: 1 Map 3 @@ -4788,7 +4790,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) input vertices: 1 Reducer 4 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -4814,9 +4816,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: @@ -4835,32 +4837,30 @@ STAGE PLANS: Reducer 4 Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -5195,41 +5195,45 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col1 - input vertices: - 1 Map 3 - Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE - HybridGraceHashJoin: true + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: string) + 0 _col0 (type: string) 1 _col0 (type: string) + outputColumnNames: _col1 input vertices: - 1 Map 4 - Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE HybridGraceHashJoin: true - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + HybridGraceHashJoin: true + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) Map 3 Map Operator Tree: TableScan diff --git ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out index 0bf92ef..081cf8c 100644 --- ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out +++ ql/src/test/results/clientpositive/tez/dynpart_sort_optimization.q.out @@ -2421,29 +2421,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: (s = 'foo') (type: boolean) - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), t (type: tinyint), i (type: int) - outputColumnNames: _col0, _col1, _col2, _col4, _col5 - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), t (type: tinyint), i (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'foo' (type: string), _col4 (type: tinyint), _col5 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: 'foo' (type: string), _col4 (type: tinyint), _col5 (type: int) - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reducer 2 Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 1974 Data size: 53304 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2497,13 +2497,13 @@ STAGE PLANS: predicate: (t = 27) (type: boolean) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), i (type: int) - outputColumnNames: _col0, _col1, _col2, _col3, _col5 + expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), 27 (type: tinyint), i (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string), 27 (type: tinyint), _col5 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: _col3 (type: string), 27 (type: tinyint), _col5 (type: int) + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reducer 2 @@ -2568,13 +2568,13 @@ STAGE PLANS: predicate: (i = 100) (type: boolean) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), t (type: tinyint) - outputColumnNames: _col0, _col1, _col2, _col3, _col4 + expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), t (type: tinyint), 100 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string), _col4 (type: tinyint), 100 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), 100 (type: int) + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) Statistics: Num rows: 429 Data size: 53255 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reducer 2 @@ -2639,13 +2639,13 @@ STAGE PLANS: predicate: ((i = 100) and (t = 27)) (type: boolean) Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string) - outputColumnNames: _col0, _col1, _col2, _col3 + expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), 27 (type: tinyint), 100 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col3 (type: string), 27 (type: tinyint), 100 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: _col3 (type: string), 27 (type: tinyint), 100 (type: int) + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reducer 2 @@ -2705,29 +2705,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((i = 100) and (s = 'foo')) (type: boolean) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), t (type: tinyint) - outputColumnNames: _col0, _col1, _col2, _col4 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), t (type: tinyint), 100 (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'foo' (type: string), _col4 (type: tinyint), 100 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: 'foo' (type: string), _col4 (type: tinyint), 100 (type: int) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reducer 2 Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2776,29 +2776,29 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((t = 27) and (s = 'foo')) (type: boolean) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: si (type: smallint), b (type: bigint), f (type: float), i (type: int) - outputColumnNames: _col0, _col1, _col2, _col5 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), 27 (type: tinyint), i (type: int) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'foo' (type: string), 27 (type: tinyint), _col5 (type: int) + key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) sort order: +++ - Map-reduce partition columns: 'foo' (type: string), 27 (type: tinyint), _col5 (type: int) - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int) + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float) Reducer 2 Reduce Operator Tree: Select Operator expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 987 Data size: 26652 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 214 Data size: 26565 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat @@ -2844,17 +2844,17 @@ STAGE PLANS: Map Operator Tree: TableScan alias: over1k - Statistics: Num rows: 3949 Data size: 106636 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 859 Data size: 106636 Basic stats: COMPLETE Column stats: NONE Filter Operator predicate: ((i = 100) and (t = 27) and (s = 'foo')) (type: boolean) - Statistics: Num rows: 493 Data size: 13312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 107 Data size: 13282 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), 27 (type: tinyint), 100 (type: int) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 - Statistics: Num rows: 493 Data size: 13312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 107 Data size: 13282 Basic stats: COMPLETE Column stats: NONE File Output Operator compressed: false - Statistics: Num rows: 493 Data size: 13312 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 107 Data size: 13282 Basic stats: COMPLETE Column stats: NONE table: input format: org.apache.hadoop.mapred.TextInputFormat output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat diff --git ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out index 97f59d9..6931398 100644 --- ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out +++ ql/src/test/results/clientpositive/tez/dynpart_sort_optimization2.q.out @@ -1613,31 +1613,31 @@ 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: _col1, _col2 + outputColumnNames: key, value Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(_col2) - keys: 'day' (type: string), _col1 (type: string) + aggregations: count(value) + keys: key (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'day' (type: string), _col1 (type: string) - sort order: ++ - Map-reduce partition columns: 'day' (type: string), _col1 (type: string) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: bigint) + value expressions: _col1 (type: bigint) Reducer 2 Execution mode: vectorized Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: 'day' (type: string), KEY._col1 (type: string) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int), 'day' (type: string) + expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int), 'day' (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1741,31 +1741,31 @@ 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: _col1, _col2 + outputColumnNames: key, value Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Group By Operator - aggregations: count(_col2) - keys: 'day' (type: string), _col1 (type: string) + aggregations: count(value) + keys: key (type: string) mode: hash - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'day' (type: string), _col1 (type: string) - sort order: ++ - Map-reduce partition columns: 'day' (type: string), _col1 (type: string) + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: bigint) + value expressions: _col1 (type: bigint) Reducer 2 Execution mode: vectorized Reduce Operator Tree: Group By Operator aggregations: count(VALUE._col0) - keys: 'day' (type: string), KEY._col1 (type: string) + keys: KEY._col0 (type: string) mode: mergepartial - outputColumnNames: _col0, _col1, _col2 + outputColumnNames: _col0, _col1 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToInteger(_col1) (type: int), UDFToInteger(_col2) (type: int), 'day' (type: string) + expressions: UDFToInteger(_col0) (type: int), UDFToInteger(_col1) (type: int), 'day' (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/tez/explainuser_1.q.out ql/src/test/results/clientpositive/tez/explainuser_1.q.out index c70f104..b614b78 100644 --- ql/src/test/results/clientpositive/tez/explainuser_1.q.out +++ ql/src/test/results/clientpositive/tez/explainuser_1.q.out @@ -1808,20 +1808,16 @@ Stage-0 SHUFFLE [RS_12] Group By Operator [GBY_11] (rows=1 width=8) Output:["_col0"],aggregations:["count('2014')"] - Merge Join Operator [MERGEJOIN_20] (rows=400 width=0) - Conds:RS_6._col0=RS_7._col0(Inner) + Merge Join Operator [MERGEJOIN_18] (rows=400 width=0) + Conds:(Inner) <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_6] - PartitionCols:_col0 Select Operator [SEL_2] (rows=20 width=184) - Output:["_col0"] TableScan [TS_0] (rows=20 width=13) default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE <-Map 4 [SIMPLE_EDGE] SHUFFLE [RS_7] - PartitionCols:_col0 Select Operator [SEL_5] (rows=20 width=184) - Output:["_col0"] TableScan [TS_3] (rows=20 width=13) default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE @@ -2120,31 +2116,31 @@ Stage-0 Stage-1 Reducer 3 File Output Operator [FS_22] - Select Operator [SEL_21] (rows=4 width=8) + Select Operator [SEL_21] (rows=2 width=8) Output:["_col0","_col1"] - Merge Join Operator [MERGEJOIN_32] (rows=4 width=8) + Merge Join Operator [MERGEJOIN_32] (rows=2 width=8) Conds:RS_18._col1=RS_19._col0(Inner),Output:["_col2","_col4"] <-Reducer 2 [SIMPLE_EDGE] SHUFFLE [RS_18] PartitionCols:_col1 - Merge Join Operator [MERGEJOIN_31] (rows=13 width=8) - Conds:RS_15._col0, 1=RS_16._col0, 1(Left Semi),Output:["_col1","_col2"] + Merge Join Operator [MERGEJOIN_31] (rows=6 width=8) + Conds:RS_15._col0, _col3=RS_16._col0, _col1(Left Semi),Output:["_col1","_col2"] <-Map 1 [SIMPLE_EDGE] SHUFFLE [RS_15] - PartitionCols:_col0, 1 + PartitionCols:_col0, _col3 Select Operator [SEL_2] (rows=17 width=16) - Output:["_col0","_col1","_col2"] + Output:["_col0","_col1","_col2","_col3"] Filter Operator [FIL_28] (rows=17 width=16) predicate:((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) TableScan [TS_0] (rows=100 width=16) default@lineitem,lineitem,Tbl:COMPLETE,Col:COMPLETE,Output:["l_orderkey","l_partkey","l_suppkey","l_linenumber"] <-Map 4 [SIMPLE_EDGE] SHUFFLE [RS_16] - PartitionCols:_col0, 1 + PartitionCols:_col0, _col1 Group By Operator [GBY_14] (rows=4 width=8) - Output:["_col0","_col1"],keys:_col0, 1 - Select Operator [SEL_5] (rows=14 width=4) - Output:["_col0"] + Output:["_col0","_col1"],keys:_col0, _col1 + Select Operator [SEL_5] (rows=14 width=8) + Output:["_col0","_col1"] Filter Operator [FIL_29] (rows=14 width=96) predicate:((l_shipmode = 'AIR') and (l_linenumber = 1) and l_orderkey is not null) TableScan [TS_3] (rows=100 width=96) @@ -2338,53 +2334,55 @@ Stage-0 limit:-1 Stage-1 Reducer 4 - File Output Operator [FS_25] - Select Operator [SEL_24] (rows=1 width=178) + File Output Operator [FS_26] + Select Operator [SEL_25] (rows=1 width=178) Output:["_col0","_col1"] <-Reducer 3 [SIMPLE_EDGE] - SHUFFLE [RS_23] - Select Operator [SEL_22] (rows=1 width=178) + SHUFFLE [RS_24] + Select Operator [SEL_23] (rows=1 width=178) Output:["_col0","_col1"] - Filter Operator [FIL_21] (rows=1 width=265) + Filter Operator [FIL_22] (rows=1 width=265) predicate:_col3 is null - Merge Join Operator [MERGEJOIN_30] (rows=404 width=265) - Conds:RS_18._col0=RS_19._col0(Left Outer),Output:["_col0","_col1","_col3"] + Merge Join Operator [MERGEJOIN_31] (rows=1 width=265) + Conds:RS_19._col0=RS_20._col0(Left Outer),Output:["_col0","_col1","_col3"] <-Map 7 [SIMPLE_EDGE] - SHUFFLE [RS_19] + SHUFFLE [RS_20] PartitionCols:_col0 - Select Operator [SEL_13] (rows=166 width=87) + Select Operator [SEL_15] (rows=1 width=87) Output:["_col0"] - Filter Operator [FIL_28] (rows=166 width=87) - predicate:(key > '2') - TableScan [TS_11] (rows=500 width=87) + Filter Operator [FIL_29] (rows=1 width=87) + predicate:false + TableScan [TS_13] (rows=500 width=87) default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] <-Reducer 2 [SIMPLE_EDGE] - SHUFFLE [RS_18] + SHUFFLE [RS_19] PartitionCols:_col0 - Merge Join Operator [MERGEJOIN_29] (rows=500 width=178) + Merge Join Operator [MERGEJOIN_30] (rows=1 width=178) Conds:(Inner),Output:["_col0","_col1"] <-Map 1 [SIMPLE_EDGE] - SHUFFLE [RS_15] - Select Operator [SEL_1] (rows=500 width=178) + SHUFFLE [RS_16] + Select Operator [SEL_2] (rows=1 width=178) Output:["_col0","_col1"] - TableScan [TS_0] (rows=500 width=178) - default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] + Filter Operator [FIL_28] (rows=1 width=178) + predicate:false + TableScan [TS_0] (rows=500 width=178) + default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"] <-Reducer 6 [SIMPLE_EDGE] - SHUFFLE [RS_16] - Select Operator [SEL_10] (rows=1 width=8) - Filter Operator [FIL_9] (rows=1 width=8) + SHUFFLE [RS_17] + Select Operator [SEL_12] (rows=1 width=8) + Filter Operator [FIL_11] (rows=1 width=8) predicate:(_col0 = 0) - Group By Operator [GBY_7] (rows=1 width=8) + Group By Operator [GBY_9] (rows=1 width=8) Output:["_col0"],aggregations:["count(VALUE._col0)"] <-Map 5 [SIMPLE_EDGE] - SHUFFLE [RS_6] - Group By Operator [GBY_5] (rows=1 width=8) + SHUFFLE [RS_8] + Group By Operator [GBY_7] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] - Select Operator [SEL_4] (rows=1 width=87) - Filter Operator [FIL_27] (rows=1 width=87) - predicate:((key > '2') and key is null) - TableScan [TS_2] (rows=500 width=87) - default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] + Filter Operator [FIL_5] (rows=1 width=4) + predicate:false + Select Operator [SEL_4] (rows=500 width=4) + TableScan [TS_3] (rows=500 width=10) + default@src_cbo,src_cbo,Tbl:COMPLETE,Col:COMPLETE PREHOOK: query: explain select p_mfgr, b.p_name, p_size from part b @@ -2414,35 +2412,35 @@ Stage-0 limit:-1 Stage-1 Reducer 3 - File Output Operator [FS_23] - Select Operator [SEL_22] (rows=1 width=223) + File Output Operator [FS_22] + Select Operator [SEL_21] (rows=1 width=223) Output:["_col0","_col1","_col2"] - Filter Operator [FIL_21] (rows=1 width=344) + Filter Operator [FIL_20] (rows=1 width=344) predicate:_col4 is null - Merge Join Operator [MERGEJOIN_28] (rows=1 width=344) - Conds:RS_18._col0, _col1=RS_19._col0, _col1(Left Outer),Output:["_col0","_col1","_col2","_col4"] + Merge Join Operator [MERGEJOIN_27] (rows=1 width=344) + Conds:RS_17._col0, _col1=RS_18._col0, _col1(Left Outer),Output:["_col0","_col1","_col2","_col4"] <-Map 6 [SIMPLE_EDGE] - SHUFFLE [RS_19] + SHUFFLE [RS_18] PartitionCols:_col0, _col1 Select Operator [SEL_13] (rows=8 width=219) Output:["_col0","_col1"] - Filter Operator [FIL_26] (rows=8 width=223) + Filter Operator [FIL_25] (rows=8 width=223) predicate:(p_size < 10) TableScan [TS_11] (rows=26 width=223) default@part,b,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_mfgr","p_size"] <-Reducer 2 [SIMPLE_EDGE] - SHUFFLE [RS_18] + SHUFFLE [RS_17] PartitionCols:_col0, _col1 - Merge Join Operator [MERGEJOIN_27] (rows=26 width=223) + Merge Join Operator [MERGEJOIN_26] (rows=26 width=223) Conds:(Inner),Output:["_col0","_col1","_col2"] <-Map 1 [SIMPLE_EDGE] - SHUFFLE [RS_15] + SHUFFLE [RS_14] Select Operator [SEL_1] (rows=26 width=223) Output:["_col0","_col1","_col2"] TableScan [TS_0] (rows=26 width=223) default@part,b,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_mfgr","p_size"] <-Reducer 5 [SIMPLE_EDGE] - SHUFFLE [RS_16] + SHUFFLE [RS_15] Select Operator [SEL_10] (rows=1 width=8) Filter Operator [FIL_9] (rows=1 width=8) predicate:(_col0 = 0) @@ -2453,7 +2451,7 @@ Stage-0 Group By Operator [GBY_5] (rows=1 width=8) Output:["_col0"],aggregations:["count()"] Select Operator [SEL_4] (rows=1 width=223) - Filter Operator [FIL_25] (rows=1 width=223) + Filter Operator [FIL_24] (rows=1 width=223) predicate:((p_size < 10) and (p_name is null or p_mfgr is null)) TableScan [TS_2] (rows=26 width=223) default@part,b,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_mfgr","p_size"] @@ -2488,30 +2486,30 @@ Stage-0 limit:-1 Stage-1 Reducer 4 - File Output Operator [FS_36] - Select Operator [SEL_35] (rows=1 width=125) + File Output Operator [FS_35] + Select Operator [SEL_34] (rows=1 width=125) Output:["_col0","_col1"] <-Reducer 3 [SIMPLE_EDGE] - SHUFFLE [RS_34] - Select Operator [SEL_33] (rows=1 width=125) + SHUFFLE [RS_33] + Select Operator [SEL_32] (rows=1 width=125) Output:["_col0","_col1"] - Filter Operator [FIL_32] (rows=1 width=133) + Filter Operator [FIL_31] (rows=1 width=133) predicate:_col3 is null - Merge Join Operator [MERGEJOIN_42] (rows=1 width=133) - Conds:RS_29.UDFToDouble(_col1)=RS_30._col0(Left Outer),Output:["_col0","_col1","_col3"] + Merge Join Operator [MERGEJOIN_41] (rows=1 width=133) + Conds:RS_28.UDFToDouble(_col1)=RS_29._col0(Left Outer),Output:["_col0","_col1","_col3"] <-Reducer 2 [SIMPLE_EDGE] - SHUFFLE [RS_29] + SHUFFLE [RS_28] PartitionCols:UDFToDouble(_col1) - Merge Join Operator [MERGEJOIN_41] (rows=26 width=125) + Merge Join Operator [MERGEJOIN_40] (rows=26 width=125) Conds:(Inner),Output:["_col0","_col1"] <-Map 1 [SIMPLE_EDGE] - SHUFFLE [RS_26] + SHUFFLE [RS_25] Select Operator [SEL_1] (rows=26 width=125) Output:["_col0","_col1"] TableScan [TS_0] (rows=26 width=125) default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_name","p_size"] <-Reducer 6 [SIMPLE_EDGE] - SHUFFLE [RS_27] + SHUFFLE [RS_26] Select Operator [SEL_17] (rows=1 width=8) Filter Operator [FIL_16] (rows=1 width=8) predicate:(_col0 = 0) @@ -2526,12 +2524,12 @@ Stage-0 SHUFFLE [RS_6] Group By Operator [GBY_5] (rows=1 width=0) Output:["_col0"],aggregations:["avg(p_size)"] - Filter Operator [FIL_38] (rows=8 width=4) + Filter Operator [FIL_37] (rows=8 width=4) predicate:(p_size < 10) TableScan [TS_2] (rows=26 width=4) default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_size"] <-Reducer 8 [SIMPLE_EDGE] - SHUFFLE [RS_30] + SHUFFLE [RS_29] PartitionCols:_col0 Group By Operator [GBY_23] (rows=1 width=8) Output:["_col0"],aggregations:["avg(VALUE._col0)"] @@ -2539,7 +2537,7 @@ Stage-0 SHUFFLE [RS_22] Group By Operator [GBY_21] (rows=1 width=0) Output:["_col0"],aggregations:["avg(p_size)"] - Filter Operator [FIL_40] (rows=8 width=4) + Filter Operator [FIL_39] (rows=8 width=4) predicate:(p_size < 10) TableScan [TS_18] (rows=26 width=4) default@part,part,Tbl:COMPLETE,Col:COMPLETE,Output:["p_size"] @@ -2580,23 +2578,23 @@ Stage-0 limit:-1 Stage-1 Reducer 5 - File Output Operator [FS_38] - Select Operator [SEL_37] (rows=1 width=106) + File Output Operator [FS_37] + Select Operator [SEL_36] (rows=1 width=106) Output:["_col0","_col1"] <-Reducer 4 [SIMPLE_EDGE] - SHUFFLE [RS_36] - Select Operator [SEL_35] (rows=1 width=106) + SHUFFLE [RS_35] + Select Operator [SEL_34] (rows=1 width=106) Output:["_col0","_col1"] - Filter Operator [FIL_34] (rows=1 width=204) + Filter Operator [FIL_33] (rows=1 width=204) predicate:_col3 is null - Merge Join Operator [MERGEJOIN_43] (rows=1 width=204) - Conds:RS_31._col0, _col1=RS_32._col0, _col1(Left Outer),Output:["_col0","_col1","_col3"] + Merge Join Operator [MERGEJOIN_42] (rows=1 width=204) + Conds:RS_30._col0, _col1=RS_31._col0, _col1(Left Outer),Output:["_col0","_col1","_col3"] <-Reducer 10 [SIMPLE_EDGE] - SHUFFLE [RS_32] + SHUFFLE [RS_31] PartitionCols:_col0, _col1 Select Operator [SEL_26] (rows=1 width=106) Output:["_col0","_col1"] - Filter Operator [FIL_40] (rows=1 width=114) + Filter Operator [FIL_39] (rows=1 width=114) predicate:((_col2 - _col1) > 600.0) Group By Operator [GBY_24] (rows=5 width=114) Output:["_col0","_col1","_col2"],aggregations:["min(VALUE._col0)","max(VALUE._col1)"],keys:KEY._col0 @@ -2605,17 +2603,15 @@ Stage-0 PartitionCols:_col0 Group By Operator [GBY_22] (rows=5 width=114) Output:["_col0","_col1","_col2"],aggregations:["min(p_retailprice)","max(p_retailprice)"],keys:p_mfgr - Select Operator [SEL_21] (rows=26 width=106) - Output:["p_mfgr","p_retailprice"] - TableScan [TS_20] (rows=26 width=106) - default@part,b,Tbl:COMPLETE,Col:COMPLETE,Output:["p_mfgr","p_retailprice"] + TableScan [TS_20] (rows=26 width=106) + default@part,b,Tbl:COMPLETE,Col:COMPLETE,Output:["p_mfgr","p_retailprice"] <-Reducer 3 [SIMPLE_EDGE] - SHUFFLE [RS_31] + SHUFFLE [RS_30] PartitionCols:_col0, _col1 - Merge Join Operator [MERGEJOIN_42] (rows=5 width=106) + Merge Join Operator [MERGEJOIN_41] (rows=5 width=106) Conds:(Inner),Output:["_col0","_col1"] <-Reducer 2 [SIMPLE_EDGE] - SHUFFLE [RS_28] + SHUFFLE [RS_27] Group By Operator [GBY_4] (rows=5 width=106) Output:["_col0","_col1"],aggregations:["min(VALUE._col0)"],keys:KEY._col0 <-Map 1 [SIMPLE_EDGE] @@ -2628,7 +2624,7 @@ Stage-0 TableScan [TS_0] (rows=26 width=106) default@part,b,Tbl:COMPLETE,Col:COMPLETE,Output:["p_mfgr","p_retailprice"] <-Reducer 8 [SIMPLE_EDGE] - SHUFFLE [RS_29] + SHUFFLE [RS_28] Select Operator [SEL_19] (rows=1 width=8) Filter Operator [FIL_18] (rows=1 width=8) predicate:(_col0 = 0) @@ -3296,17 +3292,17 @@ Stage-0 limit:-1 Stage-1 Reducer 2 - File Output Operator [FS_9] - Merge Join Operator [MERGEJOIN_10] (rows=250000 width=87) + File Output Operator [FS_8] + Merge Join Operator [MERGEJOIN_9] (rows=250000 width=87) Conds:(Inner),Output:["_col0"] <-Map 1 [SIMPLE_EDGE] - SHUFFLE [RS_5] + SHUFFLE [RS_4] Select Operator [SEL_1] (rows=500 width=87) Output:["_col0"] TableScan [TS_0] (rows=500 width=87) default@src,src,Tbl:COMPLETE,Col:COMPLETE,Output:["key"] <-Map 3 [SIMPLE_EDGE] - SHUFFLE [RS_6] + SHUFFLE [RS_5] Select Operator [SEL_3] (rows=500 width=4) TableScan [TS_2] (rows=500 width=10) default@src,src,Tbl:COMPLETE,Col:COMPLETE @@ -5874,10 +5870,10 @@ Stage-0 Select Operator [SEL_7] (rows=1 width=33) Output:["_col0","_col1","_col2"] Map Join Operator [MAPJOIN_17] (rows=1 width=33) - Conds:SEL_1.UDFToDouble(_col0)=RS_5.(UDFToDouble(_col0) + UDFToDouble(1))(Left Outer),HybridGraceHashJoin:true,Output:["_col0","_col1","_col2"] + Conds:SEL_1.UDFToDouble(_col0)=RS_5.(UDFToDouble(_col0) + 1.0)(Left Outer),HybridGraceHashJoin:true,Output:["_col0","_col1","_col2"] <-Map 4 [BROADCAST_EDGE] BROADCAST [RS_5] - PartitionCols:(UDFToDouble(_col0) + UDFToDouble(1)) + PartitionCols:(UDFToDouble(_col0) + 1.0) Select Operator [SEL_3] (rows=1 width=30) Output:["_col0"] TableScan [TS_2] (rows=1 width=30) diff --git ql/src/test/results/clientpositive/tez/explainuser_2.q.out ql/src/test/results/clientpositive/tez/explainuser_2.q.out index 5530660..cae0310 100644 --- ql/src/test/results/clientpositive/tez/explainuser_2.q.out +++ ql/src/test/results/clientpositive/tez/explainuser_2.q.out @@ -214,7 +214,7 @@ Stage-0 Select Operator [SEL_2] (rows=500 width=10) Output:["_col0"] Filter Operator [FIL_22] (rows=500 width=10) - predicate:value is not null + predicate:((11.0 = 11.0) and value is not null) TableScan [TS_0] (rows=500 width=10) default@srcpart,z,Tbl:COMPLETE,Col:NONE,Output:["value"] <-Map 4 [SIMPLE_EDGE] @@ -955,7 +955,7 @@ Stage-0 <-Select Operator [SEL_2] (rows=500 width=10) Output:["_col0"] Filter Operator [FIL_22] (rows=500 width=10) - predicate:value is not null + predicate:((11.0 = 11.0) and value is not null) TableScan [TS_0] (rows=500 width=10) default@srcpart,z,Tbl:COMPLETE,Col:NONE,Output:["value"] diff --git ql/src/test/results/clientpositive/tez/tez_self_join.q.out ql/src/test/results/clientpositive/tez/tez_self_join.q.out index f5375b2..ccd0816 100644 --- ql/src/test/results/clientpositive/tez/tez_self_join.q.out +++ ql/src/test/results/clientpositive/tez/tez_self_join.q.out @@ -42,6 +42,7 @@ POSTHOOK: type: QUERY POSTHOOK: Input: default@values__tmp__table__2 POSTHOOK: Output: default@tez_self_join2 POSTHOOK: Lineage: tez_self_join2.id1 EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: explain select s.id2, s.id3 from @@ -90,9 +91,7 @@ STAGE PLANS: outputColumnNames: _col0, _col2 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'ab' (type: string) - sort order: + - Map-reduce partition columns: 'ab' (type: string) + sort order: Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE value expressions: _col0 (type: int), _col2 (type: string) Map 4 @@ -106,9 +105,7 @@ STAGE PLANS: Select Operator Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: 'ab' (type: string) - sort order: + - Map-reduce partition columns: 'ab' (type: string) + sort order: Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Map 5 Map Operator Tree: @@ -133,16 +130,20 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 'ab' (type: string) - 1 'ab' (type: string) + 0 + 1 outputColumnNames: _col0, _col2 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: int) - sort order: + - Map-reduce partition columns: _col0 (type: int) + Select Operator + expressions: _col0 (type: int), 'ab' (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: string) + Reduce Output Operator + key expressions: _col0 (type: int) + sort order: + + Map-reduce partition columns: _col0 (type: int) + Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string), _col2 (type: string) Reducer 3 Reduce Operator Tree: Merge Join Operator @@ -151,10 +152,10 @@ STAGE PLANS: keys: 0 _col0 (type: int) 1 _col0 (type: int) - outputColumnNames: _col2 + outputColumnNames: _col1, _col2 Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: 'ab' (type: string), _col2 (type: string) + expressions: _col1 (type: string), _col2 (type: string) outputColumnNames: _col0, _col1 Statistics: Num rows: 3 Data size: 3 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -171,6 +172,7 @@ STAGE PLANS: Processor Tree: ListSink +Warning: Shuffle Join MERGEJOIN[24][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select s.id2, s.id3 from ( diff --git ql/src/test/results/clientpositive/tez/tez_union_dynamic_partition.q.out ql/src/test/results/clientpositive/tez/tez_union_dynamic_partition.q.out index abb0707..a1221a4 100644 --- ql/src/test/results/clientpositive/tez/tez_union_dynamic_partition.q.out +++ ql/src/test/results/clientpositive/tez/tez_union_dynamic_partition.q.out @@ -66,42 +66,34 @@ STAGE PLANS: alias: dummy Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: 1 (type: int) - outputColumnNames: _col0 + expressions: 1 (type: int), '2014' (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), '2014' (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - 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 - name: default.partunion1 + 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 + name: default.partunion1 Map 3 Map Operator Tree: TableScan alias: dummy Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: COMPLETE Select Operator - expressions: 2 (type: int) - outputColumnNames: _col0 + expressions: 2 (type: int), '2014' (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE - Select Operator - expressions: _col0 (type: int), '2014' (type: string) - outputColumnNames: _col0, _col1 + File Output Operator + compressed: false Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 184 Basic stats: COMPLETE Column stats: COMPLETE - 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 - name: default.partunion1 + 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 + name: default.partunion1 Union 2 Vertex: Union 2 diff --git ql/src/test/results/clientpositive/tez/vector_coalesce.q.out ql/src/test/results/clientpositive/tez/vector_coalesce.q.out index bb67008..a271bf8 100644 --- ql/src/test/results/clientpositive/tez/vector_coalesce.q.out +++ ql/src/test/results/clientpositive/tez/vector_coalesce.q.out @@ -39,16 +39,14 @@ STAGE PLANS: outputColumnNames: _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: double), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: smallint), _col5 (type: string) + key expressions: null (type: void), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: smallint), _col5 (type: string) sort order: ++++++ Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reducer 2 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: double), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: smallint), KEY.reducesinkkey5 (type: string) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: smallint), KEY.reducesinkkey5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Limit @@ -131,16 +129,14 @@ STAGE PLANS: outputColumnNames: _col1, _col2, _col3 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: tinyint), _col1 (type: double), _col2 (type: int), _col3 (type: double) + key expressions: null (type: void), _col1 (type: double), _col2 (type: int), _col3 (type: double) sort order: ++++ Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reducer 2 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: tinyint), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: double) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: double) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Limit @@ -221,16 +217,14 @@ STAGE PLANS: Select Operator Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: float), null (type: bigint), 0.0 (type: float) + key expressions: null (type: void), null (type: void), 0.0 (type: float) sort order: +++ Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reducer 2 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: float), null (type: bigint), 0.0 (type: float) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey0 (type: void), KEY.reducesinkkey2 (type: float) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Limit @@ -403,16 +397,14 @@ STAGE PLANS: Select Operator Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: float), null (type: bigint), null (type: float) + key expressions: null (type: void), null (type: void), null (type: float) sort order: +++ Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reducer 2 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: float), null (type: bigint), null (type: float) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey0 (type: void), KEY.reducesinkkey2 (type: float) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Limit @@ -481,7 +473,7 @@ STAGE PLANS: Filter Operator predicate: cbigint is null (type: boolean) Select Operator - expressions: null (type: bigint), ctinyint (type: tinyint), COALESCE(null,ctinyint) (type: bigint) + expressions: null (type: void), ctinyint (type: tinyint), COALESCE(null,ctinyint) (type: tinyint) outputColumnNames: _col0, _col1, _col2 Limit Number of rows: 10 diff --git ql/src/test/results/clientpositive/tez/vector_date_1.q.out ql/src/test/results/clientpositive/tez/vector_date_1.q.out index a27edcb..9f31820 100644 --- ql/src/test/results/clientpositive/tez/vector_date_1.q.out +++ ql/src/test/results/clientpositive/tez/vector_date_1.q.out @@ -665,7 +665,7 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 2001-01-01 (type: date), VALUE._col0 (type: date) + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: date) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 74 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/tez/vector_decimal_2.q.out ql/src/test/results/clientpositive/tez/vector_decimal_2.q.out index 701f0b7..292fb88 100644 --- ql/src/test/results/clientpositive/tez/vector_decimal_2.q.out +++ ql/src/test/results/clientpositive/tez/vector_decimal_2.q.out @@ -1048,17 +1048,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3.14 (type: decimal(4,2)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3.14 (type: decimal(3,2)) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(4,2)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3.14 (type: decimal(4,2)) + expressions: VALUE._col0 (type: decimal(4,2)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1108,17 +1111,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3.14 (type: decimal(4,2)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3.14 (type: decimal(3,2)) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(4,2)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3.14 (type: decimal(4,2)) + expressions: VALUE._col0 (type: decimal(4,2)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1168,17 +1174,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1355944339.1234567 (type: decimal(30,8)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1355944339.1234567 (type: decimal(17,7)) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(30,8)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1355944339.1234567 (type: decimal(30,8)) + expressions: VALUE._col0 (type: decimal(30,8)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1228,17 +1237,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1279,17 +1291,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1339,17 +1354,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1399,17 +1417,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1459,17 +1480,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1519,17 +1543,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1579,17 +1606,20 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: decimal(20,19)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(20,19)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1 (type: decimal(20,19)) + expressions: VALUE._col0 (type: decimal(20,19)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1649,7 +1679,7 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 0.99999999999999999999 (type: decimal(20,20)) + expressions: KEY.reducesinkkey0 (type: decimal(20,20)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator diff --git ql/src/test/results/clientpositive/tez/vector_decimal_round_2.q.out ql/src/test/results/clientpositive/tez/vector_decimal_round_2.q.out index a7b8385..3b6d42d 100644 --- ql/src/test/results/clientpositive/tez/vector_decimal_round_2.q.out +++ ql/src/test/results/clientpositive/tez/vector_decimal_round_2.q.out @@ -461,20 +461,20 @@ STAGE PLANS: alias: decimal_tbl_4_orc Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: round(pos, 9) (type: decimal(30,9)), round(neg, 9) (type: decimal(30,9)) - outputColumnNames: _col0, _col1 + expressions: round(pos, 9) (type: decimal(30,9)), round(neg, 9) (type: decimal(30,9)), 1809242.315111134 (type: decimal(17,9)), -1809242.315111134 (type: decimal(17,9)) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(30,9)) sort order: + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: decimal(30,9)) + value expressions: _col1 (type: decimal(30,9)), _col2 (type: decimal(17,9)), _col3 (type: decimal(17,9)) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: decimal(30,9)), VALUE._col0 (type: decimal(30,9)), 1809242.315111134 (type: decimal(17,9)), -1809242.315111134 (type: decimal(17,9)) + expressions: KEY.reducesinkkey0 (type: decimal(30,9)), VALUE._col0 (type: decimal(30,9)), VALUE._col1 (type: decimal(17,9)), VALUE._col2 (type: decimal(17,9)) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/tez/vector_interval_1.q.out ql/src/test/results/clientpositive/tez/vector_interval_1.q.out index dbfa842..dbf2a5a 100644 --- ql/src/test/results/clientpositive/tez/vector_interval_1.q.out +++ ql/src/test/results/clientpositive/tez/vector_interval_1.q.out @@ -162,20 +162,20 @@ STAGE PLANS: alias: vector_interval_1 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: dt (type: date), (CAST( str1 AS INTERVAL YEAR TO MONTH) + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (CAST( str1 AS INTERVAL YEAR TO MONTH) - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month) - outputColumnNames: _col0, _col2, _col3, _col5, _col6 + expressions: dt (type: date), 2-4 (type: interval_year_month), (CAST( str1 AS INTERVAL YEAR TO MONTH) + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), 0-0 (type: interval_year_month), (CAST( str1 AS INTERVAL YEAR TO MONTH) - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: date) sort order: + Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: interval_year_month), _col3 (type: interval_year_month), _col5 (type: interval_year_month), _col6 (type: interval_year_month) + value expressions: _col1 (type: interval_year_month), _col2 (type: interval_year_month), _col3 (type: interval_year_month), _col4 (type: interval_year_month), _col5 (type: interval_year_month), _col6 (type: interval_year_month) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month) + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), VALUE._col3 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out index 8409a01..064e319 100644 --- ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out +++ ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out @@ -510,18 +510,21 @@ STAGE PLANS: alias: interval_arithmetic_1 Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: -1-1 (type: interval_year_month) + outputColumnNames: _col1 Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator key expressions: 5-5 (type: interval_year_month) sort order: + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE TopN Hash Memory Usage: 0.1 + value expressions: _col1 (type: interval_year_month) Execution mode: vectorized Reducer 2 Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month) + expressions: KEY.reducesinkkey0 (type: interval_year_month), VALUE._col0 (type: interval_year_month) outputColumnNames: _col0, _col1 Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Limit diff --git ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out index 2864a48..70a402d 100644 --- ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/tez/vector_mapjoin_reduce.q.out @@ -201,15 +201,15 @@ STAGE PLANS: predicate: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean) Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int) - outputColumnNames: _col0, _col1, _col2 + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), l_linenumber (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Left Semi Join 0 to 1 keys: - 0 _col0 (type: int), 1 (type: int) - 1 _col0 (type: int), 1 (type: int) + 0 _col0 (type: int), _col3 (type: int) + 1 _col0 (type: int), _col1 (type: int) outputColumnNames: _col1, _col2 input vertices: 1 Map 2 @@ -230,18 +230,18 @@ STAGE PLANS: predicate: ((l_shipmode = 'AIR') and (l_linenumber = 1) and l_orderkey is not null) (type: boolean) Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: l_orderkey (type: int) - outputColumnNames: _col0 + expressions: l_orderkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: _col0 (type: int), 1 (type: int) + keys: _col0 (type: int), _col1 (type: int) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: _col0 (type: int), 1 (type: int) + key expressions: _col0 (type: int), _col1 (type: int) sort order: ++ - Map-reduce partition columns: _col0 (type: int), 1 (type: int) + Map-reduce partition columns: _col0 (type: int), _col1 (type: int) Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Map 3 Map Operator Tree: diff --git ql/src/test/results/clientpositive/tez/vector_null_projection.q.out ql/src/test/results/clientpositive/tez/vector_null_projection.q.out index a4ccef2..d655d78 100644 --- ql/src/test/results/clientpositive/tez/vector_null_projection.q.out +++ ql/src/test/results/clientpositive/tez/vector_null_projection.q.out @@ -120,9 +120,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator - key expressions: null (type: void) + key expressions: _col0 (type: void) sort order: + - Map-reduce partition columns: null (type: void) + Map-reduce partition columns: _col0 (type: void) Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Map 4 Map Operator Tree: @@ -139,28 +139,24 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator - key expressions: null (type: void) + key expressions: _col0 (type: void) sort order: + - Map-reduce partition columns: null (type: void) + Map-reduce partition columns: _col0 (type: void) Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reducer 3 Reduce Operator Tree: Group By Operator - keys: null (type: void) + keys: KEY._col0 (type: void) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - expressions: null (type: void) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - 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 + 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 Union 2 Vertex: Union 2 diff --git ql/src/test/results/clientpositive/tez/vector_nvl.q.out ql/src/test/results/clientpositive/tez/vector_nvl.q.out index b926ab4b..8cbfb52 100644 --- ql/src/test/results/clientpositive/tez/vector_nvl.q.out +++ ql/src/test/results/clientpositive/tez/vector_nvl.q.out @@ -21,7 +21,7 @@ STAGE PLANS: Filter Operator predicate: cdouble is null (type: boolean) Select Operator - expressions: null (type: double), 100.0 (type: double) + expressions: null (type: void), 100.0 (type: double) outputColumnNames: _col0, _col1 Limit Number of rows: 10 diff --git ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out index a790b97..a660521 100644 --- ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out +++ ql/src/test/results/clientpositive/tez/vectorized_dynamic_partition_pruning.q.out @@ -1348,12 +1348,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1375,7 +1375,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1455,9 +1455,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: @@ -1499,7 +1499,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1598,9 +1598,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Execution mode: vectorized Reducer 2 @@ -1610,7 +1610,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -1690,9 +1690,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + key expressions: (UDFToDouble(_col0) * 2.0) (type: double) sort order: + - Map-reduce partition columns: (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + Map-reduce partition columns: (UDFToDouble(_col0) * 2.0) (type: double) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: @@ -1719,7 +1719,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1813,9 +1813,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + key expressions: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) sort order: + - Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + Map-reduce partition columns: UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE Map 4 Map Operator Tree: @@ -1857,7 +1857,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 UDFToString((UDFToDouble(_col0) * UDFToDouble(2))) (type: string) + 0 UDFToString((UDFToDouble(_col0) * 2.0)) (type: string) 1 UDFToString(_col0) (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -1970,9 +1970,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reducer 2 Reduce Operator Tree: @@ -1981,7 +1981,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE Group By Operator aggregations: count() @@ -2011,32 +2011,30 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -2074,7 +2072,7 @@ POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 #### A masked pattern was here #### 1000 -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: -- non-equi join EXPLAIN select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY @@ -2171,7 +2169,7 @@ STAGE PLANS: Processor Tree: ListSink -Warning: Shuffle Join MERGEJOIN[17][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product +Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product PREHOOK: query: select count(*) from srcpart, srcpart_date_hour where (srcpart_date_hour.`date` = '2008-04-08' and srcpart_date_hour.hour = 11) and (srcpart.ds = srcpart_date_hour.ds or srcpart.hr = srcpart_date_hour.hr) PREHOOK: type: QUERY PREHOOK: Input: default@srcpart @@ -2678,17 +2676,21 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col0 (type: string) - sort order: + - Map-reduce partition columns: _col0 (type: string) + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: string) Map 5 Map Operator Tree: TableScan @@ -4247,7 +4249,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 UDFToDouble(_col0) (type: double) - 1 UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + 1 UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) input vertices: 1 Map 3 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -4275,12 +4277,12 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + key expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) sort order: + - Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + Map-reduce partition columns: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: UDFToDouble(UDFToInteger((_col0 / UDFToDouble(2)))) (type: double) + expressions: UDFToDouble(UDFToInteger((_col0 / 2.0))) (type: double) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 94 Basic stats: COMPLETE Column stats: NONE Group By Operator @@ -4367,7 +4369,7 @@ STAGE PLANS: condition map: Inner Join 0 to 1 keys: - 0 (UDFToDouble(_col0) * UDFToDouble(2)) (type: double) + 0 (UDFToDouble(_col0) * 2.0) (type: double) 1 _col0 (type: double) input vertices: 1 Map 3 @@ -4505,7 +4507,7 @@ STAGE PLANS: Inner Join 0 to 1 keys: 0 _col0 (type: string) - 1 '2008-04-08' (type: string) + 1 _col0 (type: string) input vertices: 1 Reducer 4 Statistics: Num rows: 2200 Data size: 23372 Basic stats: COMPLETE Column stats: NONE @@ -4531,9 +4533,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: '2008-04-08' (type: string) + key expressions: _col0 (type: string) sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) + Map-reduce partition columns: _col0 (type: string) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Reducer 2 Execution mode: vectorized @@ -4554,32 +4556,30 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Group By Operator - keys: '2008-04-08' (type: string) + keys: KEY._col0 (type: string) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: '2008-04-08' (type: string) - sort order: + - Map-reduce partition columns: '2008-04-08' (type: string) - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: '2008-04-08' (type: string) + Group By Operator + keys: _col0 (type: string) + mode: hash outputColumnNames: _col0 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Group By Operator - keys: _col0 (type: string) - mode: hash - outputColumnNames: _col0 + Dynamic Partitioning Event Operator + Target column: ds (string) + Target Input: srcpart + Partition key expr: ds Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Dynamic Partitioning Event Operator - Target column: ds (string) - Target Input: srcpart - Partition key expr: ds - Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Target Vertex: Map 1 + Target Vertex: Map 1 Stage: Stage-0 Fetch Operator @@ -4920,41 +4920,45 @@ STAGE PLANS: Map Operator Tree: TableScan alias: srcpart + filterExpr: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: ds (type: string), hr (type: string) - outputColumnNames: _col0, _col1 + Filter Operator + predicate: (11.0 = 11.0) (type: boolean) Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE - Map Join Operator - condition map: - Inner Join 0 to 1 - keys: - 0 _col0 (type: string) - 1 _col0 (type: string) - outputColumnNames: _col1 - input vertices: - 1 Map 3 - Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE - HybridGraceHashJoin: true + Select Operator + expressions: ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 1000 Data size: 10624 Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Inner Join 0 to 1 keys: - 0 _col1 (type: string) + 0 _col0 (type: string) 1 _col0 (type: string) + outputColumnNames: _col1 input vertices: - 1 Map 4 - Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + 1 Map 3 + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE HybridGraceHashJoin: true - Group By Operator - aggregations: count() - mode: hash - outputColumnNames: _col0 - Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - sort order: + Map Join Operator + condition map: + Inner Join 0 to 1 + keys: + 0 _col1 (type: string) + 1 _col0 (type: string) + input vertices: + 1 Map 4 + Statistics: Num rows: 1210 Data size: 12854 Basic stats: COMPLETE Column stats: NONE + HybridGraceHashJoin: true + Group By Operator + aggregations: count() + mode: hash + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE - value expressions: _col0 (type: bigint) + Reduce Output Operator + sort order: + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: bigint) Map 3 Map Operator Tree: TableScan diff --git ql/src/test/results/clientpositive/truncate_column_list_bucket.q.out ql/src/test/results/clientpositive/truncate_column_list_bucket.q.out index 39ffc6c..43574f4 100644 --- ql/src/test/results/clientpositive/truncate_column_list_bucket.q.out +++ ql/src/test/results/clientpositive/truncate_column_list_bucket.q.out @@ -118,16 +118,16 @@ STAGE PLANS: Processor Tree: TableScan alias: test_tab - Statistics: Num rows: 17 Data size: 1761 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 8 Data size: 1761 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key = '484') (type: boolean) - Statistics: Num rows: 8 Data size: 828 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 880 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: '484' (type: string), value (type: string), '1' (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 8 Data size: 828 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 880 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT * FROM test_tab WHERE part = '1' AND key = '484' @@ -196,16 +196,16 @@ STAGE PLANS: Processor Tree: TableScan alias: test_tab - Statistics: Num rows: 17 Data size: 1761 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 8 Data size: 1761 Basic stats: COMPLETE Column stats: NONE GatherStats: false Filter Operator isSamplingPred: false predicate: (key = '0') (type: boolean) - Statistics: Num rows: 8 Data size: 828 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 880 Basic stats: COMPLETE Column stats: NONE Select Operator expressions: '0' (type: string), value (type: string), '1' (type: string) outputColumnNames: _col0, _col1, _col2 - Statistics: Num rows: 8 Data size: 828 Basic stats: COMPLETE Column stats: NONE + Statistics: Num rows: 4 Data size: 880 Basic stats: COMPLETE Column stats: NONE ListSink PREHOOK: query: SELECT * FROM test_tab WHERE part = '1' AND key = '0' diff --git ql/src/test/results/clientpositive/udf_folder_constants.q.out ql/src/test/results/clientpositive/udf_folder_constants.q.out index 3e765d9..b26b16c 100644 --- ql/src/test/results/clientpositive/udf_folder_constants.q.out +++ ql/src/test/results/clientpositive/udf_folder_constants.q.out @@ -62,14 +62,15 @@ STAGE PLANS: predicate: ((year = 2001) and month is not null) (type: boolean) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: month (type: int) - outputColumnNames: _col1 + expressions: 2001 (type: int), month (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col1 (type: int) sort order: + Map-reduce partition columns: _col1 (type: int) Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: int) TableScan alias: b Statistics: Num rows: 1 Data size: 2 Basic stats: COMPLETE Column stats: NONE @@ -92,9 +93,10 @@ STAGE PLANS: keys: 0 _col1 (type: int) 1 _col0 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: 978336000 (type: bigint) + expressions: to_unix_timestamp(concat(_col0, '-01-01 00:00:00')) (type: bigint) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 7 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/union_view.q.out ql/src/test/results/clientpositive/union_view.q.out index badd209..cfe3bed 100644 --- ql/src/test/results/clientpositive/union_view.q.out +++ ql/src/test/results/clientpositive/union_view.q.out @@ -357,22 +357,18 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 TableScan alias: src_union_2 filterExpr: ((key = 86) and (ds = '1')) (type: boolean) @@ -381,22 +377,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '1')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 TableScan alias: src_union_3 filterExpr: ((key = 86) and (ds = '1')) (type: boolean) @@ -405,22 +397,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '1')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '1' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 @@ -470,22 +458,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '2')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '2' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 TableScan alias: src_union_2 filterExpr: ((key = 86) and (ds = '2')) (type: boolean) @@ -494,22 +478,18 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '2' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 TableScan alias: src_union_3 filterExpr: ((key = 86) and (ds = '2')) (type: boolean) @@ -518,22 +498,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '2')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '2' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 @@ -583,22 +559,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '3')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '3' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 TableScan alias: src_union_2 filterExpr: ((key = 86) and (ds = '3')) (type: boolean) @@ -607,22 +579,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '3')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '3' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 TableScan alias: src_union_3 filterExpr: ((key = 86) and (ds = '3')) (type: boolean) @@ -631,22 +599,18 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '3' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '3' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 502 Data size: 5312 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 502 Data size: 5312 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 + 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 @@ -700,20 +664,16 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), ds (type: string) - outputColumnNames: _col1, _col2 + expressions: 86 (type: int), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col1 (type: string), _col2 (type: string) - outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col2 (type: string) - sort order: + - Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) TableScan alias: src_union_2 filterExpr: ((key = 86) and ds is not null) (type: boolean) @@ -722,20 +682,16 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), ds (type: string) - outputColumnNames: _col1, _col2 + expressions: 86 (type: int), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col1 (type: string), _col2 (type: string) - outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col2 (type: string) - sort order: + - Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) TableScan alias: src_union_3 filterExpr: ((key = 86) and ds is not null) (type: boolean) @@ -744,23 +700,19 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string), ds (type: string) - outputColumnNames: _col1, _col2 + expressions: 86 (type: int), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: _col1 (type: string), _col2 (type: string) - outputColumnNames: _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - Reduce Output Operator - key expressions: _col2 (type: string) - sort order: + - Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: string) + value expressions: _col0 (type: int), _col1 (type: string) Reduce Operator Tree: Select Operator - expressions: 86 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string) + expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY.reducesinkkey0 (type: string) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1250 Data size: 13280 Basic stats: COMPLETE Column stats: NONE File Output Operator @@ -1225,22 +1177,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '4')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '4' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 TableScan alias: src_union_2 filterExpr: ((key = 86) and (ds = '4')) (type: boolean) @@ -1249,22 +1197,18 @@ STAGE PLANS: predicate: ((key = 86) and (ds = '4')) (type: boolean) Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE Union Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '4' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 TableScan alias: src_union_3 filterExpr: ((key = 86) and (ds = '4')) (type: boolean) @@ -1273,22 +1217,18 @@ STAGE PLANS: predicate: (key = 86) (type: boolean) Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: value (type: string) - outputColumnNames: _col1 + expressions: 86 (type: int), value (type: string), '4' (type: string) + outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 86 (type: int), _col1 (type: string), '4' (type: string) - outputColumnNames: _col0, _col1, _col2 + File Output Operator + compressed: false Statistics: Num rows: 252 Data size: 2656 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 252 Data size: 2656 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 + 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 diff --git ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out index ba3a0b8..5e166e6 100644 --- ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out +++ ql/src/test/results/clientpositive/unionall_unbalancedppd.q.out @@ -386,20 +386,18 @@ STAGE PLANS: predicate: (f1 = 1) (type: boolean) Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: 1 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 3 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 1 (type: int) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 4 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 + 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 TableScan alias: union_all_bug_test_2 Statistics: Num rows: 2 Data size: 2 Basic stats: COMPLETE Column stats: NONE @@ -407,20 +405,18 @@ STAGE PLANS: predicate: (f1 = 1) (type: boolean) Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE Select Operator + expressions: 1 (type: int) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE Union Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: NONE - Select Operator - expressions: 1 (type: int) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 2 Data size: 4 Basic stats: COMPLETE Column stats: NONE - File Output Operator - compressed: false - Statistics: Num rows: 2 Data size: 4 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 + 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 diff --git ql/src/test/results/clientpositive/vector_coalesce.q.out ql/src/test/results/clientpositive/vector_coalesce.q.out index e126dcb..3710e76 100644 --- ql/src/test/results/clientpositive/vector_coalesce.q.out +++ ql/src/test/results/clientpositive/vector_coalesce.q.out @@ -33,14 +33,13 @@ STAGE PLANS: outputColumnNames: _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: double), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: smallint), _col5 (type: string) + key expressions: null (type: void), _col1 (type: string), _col2 (type: int), _col3 (type: float), _col4 (type: smallint), _col5 (type: string) sort order: ++++++ Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: double), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: smallint), KEY.reducesinkkey5 (type: string) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey1 (type: string), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: float), KEY.reducesinkkey4 (type: smallint), KEY.reducesinkkey5 (type: string) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Limit @@ -117,14 +116,13 @@ STAGE PLANS: outputColumnNames: _col1, _col2, _col3 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: tinyint), _col1 (type: double), _col2 (type: int), _col3 (type: double) + key expressions: null (type: void), _col1 (type: double), _col2 (type: int), _col3 (type: double) sort order: ++++ Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: tinyint), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: double) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey1 (type: double), KEY.reducesinkkey2 (type: int), KEY.reducesinkkey3 (type: double) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Limit @@ -199,14 +197,13 @@ STAGE PLANS: Select Operator Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: float), null (type: bigint), 0.0 (type: float) + key expressions: null (type: void), null (type: void), 0.0 (type: float) sort order: +++ Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: float), null (type: bigint), 0.0 (type: float) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey0 (type: void), KEY.reducesinkkey2 (type: float) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Limit @@ -365,14 +362,13 @@ STAGE PLANS: Select Operator Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator - key expressions: null (type: float), null (type: bigint), null (type: float) + key expressions: null (type: void), null (type: void), null (type: float) sort order: +++ Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE TopN Hash Memory Usage: 0.1 - Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: null (type: float), null (type: bigint), null (type: float) + expressions: KEY.reducesinkkey0 (type: void), KEY.reducesinkkey0 (type: void), KEY.reducesinkkey2 (type: float) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 3072 Data size: 660491 Basic stats: COMPLETE Column stats: NONE Limit @@ -443,7 +439,7 @@ STAGE PLANS: predicate: cbigint is null (type: boolean) Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: null (type: bigint), ctinyint (type: tinyint), COALESCE(null,ctinyint) (type: bigint) + expressions: null (type: void), ctinyint (type: tinyint), COALESCE(null,ctinyint) (type: tinyint) outputColumnNames: _col0, _col1, _col2 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Limit @@ -456,7 +452,6 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Execution mode: vectorized Stage: Stage-0 Fetch Operator diff --git ql/src/test/results/clientpositive/vector_date_1.q.out ql/src/test/results/clientpositive/vector_date_1.q.out index da608bf..a394e0f 100644 --- ql/src/test/results/clientpositive/vector_date_1.q.out +++ ql/src/test/results/clientpositive/vector_date_1.q.out @@ -617,7 +617,7 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 2001-01-01 (type: date), VALUE._col0 (type: date) + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: date) outputColumnNames: _col0, _col1 Statistics: Num rows: 1 Data size: 74 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/vector_decimal_2.q.out ql/src/test/results/clientpositive/vector_decimal_2.q.out index 5e5b36c..892a1b3 100644 --- ql/src/test/results/clientpositive/vector_decimal_2.q.out +++ ql/src/test/results/clientpositive/vector_decimal_2.q.out @@ -914,15 +914,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3.14 (type: decimal(4,2)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3.14 (type: decimal(3,2)) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(4,2)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3.14 (type: decimal(4,2)) + expressions: VALUE._col0 (type: decimal(4,2)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -966,15 +969,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3.14 (type: decimal(4,2)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3.14 (type: decimal(3,2)) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(4,2)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3.14 (type: decimal(4,2)) + expressions: VALUE._col0 (type: decimal(4,2)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1018,15 +1024,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1355944339.1234567 (type: decimal(30,8)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1355944339.1234567 (type: decimal(17,7)) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(30,8)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1355944339.1234567 (type: decimal(30,8)) + expressions: VALUE._col0 (type: decimal(30,8)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1070,15 +1079,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1113,15 +1125,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1165,15 +1180,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1217,15 +1235,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1269,15 +1290,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1321,15 +1345,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 3 (type: decimal(10,0)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 3 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(10,0)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 3 (type: decimal(10,0)) + expressions: VALUE._col0 (type: decimal(10,0)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1373,15 +1400,18 @@ STAGE PLANS: alias: decimal_2 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: 1 (type: decimal(20,19)) + outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE Reduce Output Operator key expressions: 1 (type: int) sort order: + Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE + value expressions: _col0 (type: decimal(20,19)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 1 (type: decimal(20,19)) + expressions: VALUE._col0 (type: decimal(20,19)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator @@ -1433,7 +1463,7 @@ STAGE PLANS: Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 0.99999999999999999999 (type: decimal(20,20)) + expressions: KEY.reducesinkkey0 (type: decimal(20,20)) outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE File Output Operator diff --git ql/src/test/results/clientpositive/vector_decimal_round_2.q.out ql/src/test/results/clientpositive/vector_decimal_round_2.q.out index 4924bff..5b321c2 100644 --- ql/src/test/results/clientpositive/vector_decimal_round_2.q.out +++ ql/src/test/results/clientpositive/vector_decimal_round_2.q.out @@ -431,18 +431,18 @@ STAGE PLANS: alias: decimal_tbl_4_orc Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: round(pos, 9) (type: decimal(30,9)), round(neg, 9) (type: decimal(30,9)) - outputColumnNames: _col0, _col1 + expressions: round(pos, 9) (type: decimal(30,9)), round(neg, 9) (type: decimal(30,9)), 1809242.315111134 (type: decimal(17,9)), -1809242.315111134 (type: decimal(17,9)) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: decimal(30,9)) sort order: + Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE - value expressions: _col1 (type: decimal(30,9)) + value expressions: _col1 (type: decimal(30,9)), _col2 (type: decimal(17,9)), _col3 (type: decimal(17,9)) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: decimal(30,9)), VALUE._col0 (type: decimal(30,9)), 1809242.315111134 (type: decimal(17,9)), -1809242.315111134 (type: decimal(17,9)) + expressions: KEY.reducesinkkey0 (type: decimal(30,9)), VALUE._col0 (type: decimal(30,9)), VALUE._col1 (type: decimal(17,9)), VALUE._col2 (type: decimal(17,9)) outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 1 Data size: 224 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/vector_interval_1.q.out ql/src/test/results/clientpositive/vector_interval_1.q.out index 6845628..379747c 100644 --- ql/src/test/results/clientpositive/vector_interval_1.q.out +++ ql/src/test/results/clientpositive/vector_interval_1.q.out @@ -148,18 +148,18 @@ STAGE PLANS: alias: vector_interval_1 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: dt (type: date), (CAST( str1 AS INTERVAL YEAR TO MONTH) + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (CAST( str1 AS INTERVAL YEAR TO MONTH) - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month) - outputColumnNames: _col0, _col2, _col3, _col5, _col6 + expressions: dt (type: date), 2-4 (type: interval_year_month), (CAST( str1 AS INTERVAL YEAR TO MONTH) + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 + CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), 0-0 (type: interval_year_month), (CAST( str1 AS INTERVAL YEAR TO MONTH) - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month), (1-2 - CAST( str1 AS INTERVAL YEAR TO MONTH)) (type: interval_year_month) + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE Reduce Output Operator key expressions: _col0 (type: date) sort order: + Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE - value expressions: _col2 (type: interval_year_month), _col3 (type: interval_year_month), _col5 (type: interval_year_month), _col6 (type: interval_year_month) + value expressions: _col1 (type: interval_year_month), _col2 (type: interval_year_month), _col3 (type: interval_year_month), _col4 (type: interval_year_month), _col5 (type: interval_year_month), _col6 (type: interval_year_month) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: KEY.reducesinkkey0 (type: date), 2-4 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), 0-0 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month) + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: interval_year_month), VALUE._col1 (type: interval_year_month), VALUE._col2 (type: interval_year_month), VALUE._col3 (type: interval_year_month), VALUE._col4 (type: interval_year_month), VALUE._col5 (type: interval_year_month) outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6 Statistics: Num rows: 2 Data size: 442 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out index cd8111d..bad7e4a 100644 --- ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out +++ ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out @@ -480,16 +480,19 @@ STAGE PLANS: alias: interval_arithmetic_1 Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE Select Operator + expressions: -1-1 (type: interval_year_month) + outputColumnNames: _col1 Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator key expressions: 5-5 (type: interval_year_month) sort order: + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE TopN Hash Memory Usage: 0.1 + value expressions: _col1 (type: interval_year_month) Execution mode: vectorized Reduce Operator Tree: Select Operator - expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month) + expressions: KEY.reducesinkkey0 (type: interval_year_month), VALUE._col0 (type: interval_year_month) outputColumnNames: _col0, _col1 Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Limit diff --git ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out index 20f79c1..b37d1fc 100644 --- ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out +++ ql/src/test/results/clientpositive/vector_mapjoin_reduce.q.out @@ -469,18 +469,18 @@ STAGE PLANS: predicate: ((l_shipmode = 'AIR') and (l_linenumber = 1) and l_orderkey is not null) (type: boolean) Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: l_orderkey (type: int) - outputColumnNames: _col0 + expressions: l_orderkey (type: int), 1 (type: int) + outputColumnNames: _col0, _col1 Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE Group By Operator - keys: _col0 (type: int), 1 (type: int) + keys: _col0 (type: int), _col1 (type: int) mode: hash outputColumnNames: _col0, _col1 Statistics: Num rows: 25 Data size: 2999 Basic stats: COMPLETE Column stats: NONE HashTable Sink Operator keys: - 0 _col0 (type: int), 1 (type: int) - 1 _col0 (type: int), 1 (type: int) + 0 _col0 (type: int), _col3 (type: int) + 1 _col0 (type: int), _col1 (type: int) Stage: Stage-8 Map Reduce @@ -492,15 +492,15 @@ STAGE PLANS: predicate: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean) Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int) - outputColumnNames: _col0, _col1, _col2 + expressions: l_orderkey (type: int), l_partkey (type: int), l_suppkey (type: int), l_linenumber (type: int) + outputColumnNames: _col0, _col1, _col2, _col3 Statistics: Num rows: 50 Data size: 5999 Basic stats: COMPLETE Column stats: NONE Map Join Operator condition map: Left Semi Join 0 to 1 keys: - 0 _col0 (type: int), 1 (type: int) - 1 _col0 (type: int), 1 (type: int) + 0 _col0 (type: int), _col3 (type: int) + 1 _col0 (type: int), _col1 (type: int) outputColumnNames: _col1, _col2 Statistics: Num rows: 55 Data size: 6598 Basic stats: COMPLETE Column stats: NONE File Output Operator diff --git ql/src/test/results/clientpositive/vector_null_projection.q.out ql/src/test/results/clientpositive/vector_null_projection.q.out index 7517cc2..6daef59 100644 --- ql/src/test/results/clientpositive/vector_null_projection.q.out +++ ql/src/test/results/clientpositive/vector_null_projection.q.out @@ -111,9 +111,9 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator - key expressions: null (type: void) + key expressions: _col0 (type: void) sort order: + - Map-reduce partition columns: null (type: void) + Map-reduce partition columns: _col0 (type: void) Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE TableScan alias: b @@ -130,27 +130,23 @@ STAGE PLANS: outputColumnNames: _col0 Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Output Operator - key expressions: null (type: void) + key expressions: _col0 (type: void) sort order: + - Map-reduce partition columns: null (type: void) + Map-reduce partition columns: _col0 (type: void) Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE Reduce Operator Tree: Group By Operator - keys: null (type: void) + keys: KEY._col0 (type: void) mode: mergepartial outputColumnNames: _col0 Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - Select Operator - expressions: null (type: void) - outputColumnNames: _col0 + File Output Operator + compressed: false Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - File Output Operator - compressed: false - Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE - 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 + 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 diff --git ql/src/test/results/clientpositive/vector_nvl.q.out ql/src/test/results/clientpositive/vector_nvl.q.out index 69712aa..8c3e776 100644 --- ql/src/test/results/clientpositive/vector_nvl.q.out +++ ql/src/test/results/clientpositive/vector_nvl.q.out @@ -23,7 +23,7 @@ STAGE PLANS: predicate: cdouble is null (type: boolean) Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Select Operator - expressions: null (type: double), 100.0 (type: double) + expressions: null (type: void), 100.0 (type: double) outputColumnNames: _col0, _col1 Statistics: Num rows: 6144 Data size: 1320982 Basic stats: COMPLETE Column stats: NONE Limit @@ -36,7 +36,6 @@ STAGE PLANS: input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - Execution mode: vectorized Stage: Stage-0 Fetch Operator