diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties index a081638..e2305d8 100644 --- a/itests/src/test/resources/testconfiguration.properties +++ b/itests/src/test/resources/testconfiguration.properties @@ -1433,6 +1433,7 @@ miniSparkOnYarn.only.query.files=spark_combine_equivalent_work.q,\ spark_dynamic_partition_pruning.q,\ spark_dynamic_partition_pruning_2.q,\ spark_dynamic_partition_pruning_3.q,\ + spark_dynamic_partition_pruning_4.q,\ spark_dynamic_partition_pruning_mapjoin_only.q,\ spark_constprog_dpp.q,\ spark_dynamic_partition_pruning_recursive_mapjoin.q,\ diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java index 2d3d756..e8c019a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkDynamicPartitionPruner.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.ObjectInputStream; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -124,18 +125,23 @@ private void processFiles(MapWork work, JobConf jobConf) throws HiveException { for (FileStatus fstatus : fs.listStatus(sourceDir)) { LOG.info("Start processing pruning file: " + fstatus.getPath()); in = new ObjectInputStream(fs.open(fstatus.getPath())); - String columnName = in.readUTF(); + final int numName = in.readInt(); SourceInfo info = null; + Set columnNames = new HashSet<>(); + for (int i = 0; i < numName; i++) { + columnNames.add(in.readUTF()); + } for (SourceInfo si : sourceInfoMap.get(name)) { - if (columnName.equals(si.columnName)) { + if (columnNames.contains(si.columnName)) { info = si; break; } } Preconditions.checkArgument(info != null, - "AssertionError: no source info for the column: " + columnName); + "AssertionError: no source info for the column: " + + Arrays.toString(columnNames.toArray())); // Read fields while (in.available() > 0) { @@ -172,7 +178,8 @@ private void prunePartitions(MapWork work) throws HiveException { private void prunePartitionSingleSource(SourceInfo info, MapWork work) throws HiveException { Set values = info.values; - String columnName = info.columnName; + // strip the column name of the targetId + String columnName = info.columnName.substring(info.columnName.indexOf(':') + 1); ObjectInspector oi = PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(TypeInfoFactory diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java index 4d5c234..97e8ceb 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/SparkDynamicPartitionPruningResolver.java @@ -91,7 +91,7 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) throws // For each SparkPartitionPruningSinkOperator, take the target MapWork and see if it is in a dependent SparkTask for (Operator op : pruningSinkOps) { SparkPartitionPruningSinkOperator pruningSinkOp = (SparkPartitionPruningSinkOperator) op; - MapWork targetMapWork = pruningSinkOp.getConf().getTargetMapWork(); + MapWork targetMapWork = pruningSinkOp.getConf().getTargetMapWorks().get(0); // Check if the given SparkTask has a child SparkTask that contains the target MapWork // If it does not, then remove the DPP op @@ -120,9 +120,9 @@ private void removeSparkPartitionPruningSink(BaseWork sourceWork, MapWork target String sourceWorkId = SparkUtilities.getWorkId(sourceWork); SparkPartitionPruningSinkDesc pruningSinkDesc = pruningSinkOp.getConf(); targetMapWork.getEventSourceTableDescMap().get(sourceWorkId).remove(pruningSinkDesc.getTable()); - targetMapWork.getEventSourceColumnNameMap().get(sourceWorkId).remove(pruningSinkDesc.getTargetColumnName()); - targetMapWork.getEventSourceColumnTypeMap().get(sourceWorkId).remove(pruningSinkDesc.getTargetColumnType()); - targetMapWork.getEventSourcePartKeyExprMap().get(sourceWorkId).remove(pruningSinkDesc.getPartKey()); + targetMapWork.getEventSourceColumnNameMap().get(sourceWorkId).remove(pruningSinkDesc.getTargetColumnName().get(0)); + targetMapWork.getEventSourceColumnTypeMap().get(sourceWorkId).remove(pruningSinkDesc.getTargetColumnType().get(0)); + targetMapWork.getEventSourcePartKeyExprMap().get(sourceWorkId).remove(pruningSinkDesc.getPartKey().get(0)); } /** diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java index 2641c1a..e4760a1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/CombineEquivalentWorkResolver.java @@ -35,8 +35,10 @@ import org.apache.hadoop.hive.ql.exec.Task; import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities; import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator; +import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.ql.plan.TableDesc; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hive.ql.exec.MapJoinOperator; @@ -82,13 +84,14 @@ public int compare(BaseWork o1, BaseWork o2) { } }; + // maps from a work to the DPPs it contains + private Map> workToDpps = new HashMap<>(); + @Override public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) throws SemanticException { if (nd instanceof SparkTask) { SparkTask sparkTask = (SparkTask) nd; SparkWork sparkWork = sparkTask.getWork(); - Set roots = sparkWork.getRoots(); - compareWorksRecursively(roots, sparkWork); // For dpp case, dpp sink will appear in Task1 and the target work of dpp sink will appear in Task2. // Task2 is the child task of Task1. Task2 will be traversed before task1 because TaskGraphWalker will first // put children task in the front of task queue. @@ -100,11 +103,15 @@ public Object dispatch(Node nd, Stack stack, Object... nodeOutputs) throws removeEmptySparkTask(sparkTask); } } + + Set roots = sparkWork.getRoots(); + compareWorksRecursively(roots, sparkWork); } return null; } private void compareWorksRecursively(Set works, SparkWork sparkWork) { + workToDpps.clear(); // find out all equivalent works in the Set. Set> equivalentWorks = compareChildWorks(works, sparkWork); // combine equivalent work into single one in SparkWork's work graph. @@ -154,14 +161,75 @@ private boolean belongToSet(Set set, BaseWork work, SparkWork sparkWor return false; } + // merge the second into the first + private void combineEquivalentDPPSinks(SparkPartitionPruningSinkDesc first, + SparkPartitionPruningSinkDesc second, String firstId, String secondId) { + MapWork target1 = first.getTargetMapWorks().get(0); + MapWork target2 = second.getTargetMapWorks().get(0); + + first.getTargetMapWorks().add(target2); + first.getPartKey().add(second.getPartKey().get(0)); + first.getTargetColumnType().add(second.getTargetColumnType().get(0)); + first.getTargetColumnName().add(second.getTargetColumnName().get(0)); + + // update target map works + target2.setTmpPathForPartitionPruning(target1.getTmpPathForPartitionPruning()); + + List partKey = target2.getEventSourcePartKeyExprMap().get(secondId); + partKey.remove(second.getPartKey().get(0)); + if (partKey.isEmpty()) { + target2.getEventSourcePartKeyExprMap().remove(secondId); + } + List newPartKey = target2.getEventSourcePartKeyExprMap().computeIfAbsent( + firstId, v -> new ArrayList<>()); + newPartKey.add(second.getPartKey().get(0)); + + List tableDesc = target2.getEventSourceTableDescMap().get(secondId); + tableDesc.remove(second.getTable()); + if (tableDesc.isEmpty()) { + target2.getEventSourceTableDescMap().remove(secondId); + } + List newTableDesc = target2.getEventSourceTableDescMap().computeIfAbsent( + firstId, v -> new ArrayList<>()); + newTableDesc.add(second.getTable()); + + List columnName = target2.getEventSourceColumnNameMap().get(secondId); + columnName.remove(second.getTargetColumnName().get(0)); + if (columnName.isEmpty()) { + target2.getEventSourceColumnNameMap().remove(secondId); + } + List newColumnName = target2.getEventSourceColumnNameMap().computeIfAbsent( + firstId, v -> new ArrayList<>()); + newColumnName.add(second.getTargetColumnName().get(0)); + + List columnType = target2.getEventSourceColumnTypeMap().get(secondId); + columnType.remove(second.getTargetColumnType().get(0)); + if (columnType.isEmpty()) { + target2.getEventSourceColumnTypeMap().remove(secondId); + } + List newColumnType = target2.getEventSourceColumnTypeMap().computeIfAbsent( + firstId, v -> new ArrayList<>()); + newColumnType.add(second.getTargetColumnType().get(0)); + } + private Set combineEquivalentWorks(Set> equivalentWorks, SparkWork sparkWork) { Set removedWorks = Sets.newHashSet(); for (Set workSet : equivalentWorks) { if (workSet.size() > 1) { Iterator iterator = workSet.iterator(); BaseWork first = iterator.next(); + List dppList1 = workToDpps.get(first); + String firstId = SparkUtilities.getWorkId(first); while (iterator.hasNext()) { BaseWork next = iterator.next(); + if (dppList1 != null) { + List dppList2 = workToDpps.get(next); + // equivalent works must have dpp lists of same size + for (int i = 0; i < dppList1.size(); i++) { + combineEquivalentDPPSinks(dppList1.get(i), dppList2.get(i), + firstId, SparkUtilities.getWorkId(next)); + } + } replaceWork(next, first, sparkWork); removedWorks.add(next); } @@ -231,7 +299,14 @@ private boolean compareWork(BaseWork first, BaseWork second, SparkWork sparkWork // leave work's output may be read in further SparkWork/FetchWork, we should not combine // leave works without notifying further SparkWork/FetchWork. if (sparkWork.getLeaves().contains(first) && sparkWork.getLeaves().contains(second)) { - return false; + Set> leafOps = first.getAllLeafOperators(); + leafOps.addAll(second.getAllLeafOperators()); + for (Operator operator : leafOps) { + // we know how to handle DPP sinks + if (!(operator instanceof SparkPartitionPruningSinkOperator)) { + return false; + } + } } // need to check paths and partition desc for MapWorks @@ -248,9 +323,10 @@ private boolean compareWork(BaseWork first, BaseWork second, SparkWork sparkWork Iterator> firstIterator = firstRootOperators.iterator(); Iterator> secondIterator = secondRootOperators.iterator(); while (firstIterator.hasNext()) { - boolean result = compareOperatorChain(firstIterator.next(), secondIterator.next()); + boolean result = compareOperatorChain(firstIterator.next(), secondIterator.next(), + first, second); if (!result) { - return result; + return false; } } @@ -290,10 +366,11 @@ private boolean hasSameParent(BaseWork first, BaseWork second, SparkWork sparkWo return result; } - private boolean compareOperatorChain(Operator firstOperator, Operator secondOperator) { + private boolean compareOperatorChain(Operator firstOperator, Operator secondOperator, + BaseWork first, BaseWork second) { boolean result = compareCurrentOperator(firstOperator, secondOperator); if (!result) { - return result; + return false; } List> firstOperatorChildOperators = firstOperator.getChildOperators(); @@ -302,19 +379,26 @@ private boolean compareOperatorChain(Operator firstOperator, Operator seco return false; } else if (firstOperatorChildOperators != null && secondOperatorChildOperators == null) { return false; - } else if (firstOperatorChildOperators != null && secondOperatorChildOperators != null) { + } else if (firstOperatorChildOperators != null) { if (firstOperatorChildOperators.size() != secondOperatorChildOperators.size()) { return false; } int size = firstOperatorChildOperators.size(); for (int i = 0; i < size; i++) { - result = compareOperatorChain(firstOperatorChildOperators.get(i), secondOperatorChildOperators.get(i)); + result = compareOperatorChain(firstOperatorChildOperators.get(i), + secondOperatorChildOperators.get(i), first, second); if (!result) { return false; } } } + if (firstOperator instanceof SparkPartitionPruningSinkOperator) { + List dpps = workToDpps.computeIfAbsent(first, k -> new ArrayList<>()); + dpps.add(((SparkPartitionPruningSinkOperator) firstOperator).getConf()); + dpps = workToDpps.computeIfAbsent(second, k -> new ArrayList<>()); + dpps.add(((SparkPartitionPruningSinkOperator) secondOperator).getConf()); + } return true; } @@ -347,9 +431,9 @@ private void removeDynamicPartitionPruningSink(List removedMapWorkList, SparkUtilities.collectOp(pruningList, root, SparkPartitionPruningSinkOperator.class); for (Operator pruneSinkOp : pruningList) { SparkPartitionPruningSinkOperator sparkPruneSinkOp = (SparkPartitionPruningSinkOperator) pruneSinkOp; - if (removedMapWorkList.contains(sparkPruneSinkOp.getConf().getTargetWork())) { + if (removedMapWorkList.containsAll(sparkPruneSinkOp.getConf().getTargetMapWorkNames())) { LOG.debug("ready to remove the sparkPruneSinkOp which target work is " + - sparkPruneSinkOp.getConf().getTargetWork() + " because the MapWork is equals to other map work and " + + sparkPruneSinkOp.getConf().getTargetWorks() + " because the MapWork is equals to other map work and " + "has been deleted!"); // If there is branch, remove prune sink operator branch in the baseWork // If there is no branch, remove the whole baseWork diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java index baf85cf..0a21c96 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkPartitionPruningSinkDesc.java @@ -18,33 +18,40 @@ package org.apache.hadoop.hive.ql.optimizer.spark; +import com.google.common.base.Preconditions; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.exec.TableScanOperator; import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc; import org.apache.hadoop.hive.ql.plan.Explain; import org.apache.hadoop.hive.ql.plan.ExprNodeDesc; import org.apache.hadoop.hive.ql.plan.MapWork; +import org.apache.hadoop.hive.ql.plan.OperatorDesc; import org.apache.hadoop.hive.ql.plan.TableDesc; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; + @Explain(displayName = "Spark Partition Pruning Sink Operator") public class SparkPartitionPruningSinkDesc extends AbstractOperatorDesc { // column in the target table that will be pruned against - private String targetColumnName; + private List targetColumnName = new ArrayList<>(); // type of target column - private String targetColumnType; + private List targetColumnType = new ArrayList<>(); private TableDesc table; private transient TableScanOperator tableScan; // the partition column we're interested in - private ExprNodeDesc partKey; + private List partKey = new ArrayList<>(); private Path path; - private MapWork targetMapWork; + private List targetMapWorks = new LinkedList<>(); @Explain(displayName = "tmp Path", explainLevels = { Explain.Level.EXTENDED }) public Path getPath() { @@ -55,17 +62,26 @@ public void setPath(Path path) { this.path = path; } - @Explain(displayName = "target work") - public String getTargetWork() { - return this.targetMapWork.getName(); + @Explain(displayName = "target works") + public String getTargetWorks() { + String[] targets = new String[targetMapWorks.size()]; + int i = 0; + for (MapWork work : targetMapWorks) { + targets[i++] = work.getName(); + } + return Arrays.toString(targets); } - public MapWork getTargetMapWork() { - return this.targetMapWork; + public List getTargetMapWorks() { + return targetMapWorks; } - public void setTargetMapWork(MapWork targetMapWork) { - this.targetMapWork = targetMapWork; + public List getTargetMapWorkNames() { + List res = new ArrayList<>(targetMapWorks.size()); + for (MapWork mapWork : targetMapWorks) { + res.add(mapWork.getName()); + } + return res; } public TableScanOperator getTableScan() { @@ -78,31 +94,50 @@ public void setTableScan(TableScanOperator tableScan) { @Explain(displayName = "Target column") public String displayTargetColumn() { - return targetColumnName + " (" + targetColumnType + ")"; + String[] res = new String[targetColumnName.size()]; + for (int i = 0; i < res.length; i++) { + res[i] = targetColumnName.get(i) + " (" + targetColumnType.get(i) + ")"; + } + return Arrays.toString(res); } - public String getTargetColumnName() { + public List getTargetColumnName() { return targetColumnName; } - public void setTargetColumnName(String targetColumnName) { - this.targetColumnName = targetColumnName; + public void setTargetColumnName(String columnName) { + Preconditions.checkState(targetColumnName.size() < 2, "The DPP contains multiple targets."); + if (targetColumnName.isEmpty()) { + targetColumnName.add(columnName); + } else { + targetColumnName.set(0, columnName); + } } - public String getTargetColumnType() { + public List getTargetColumnType() { return targetColumnType; } public void setTargetColumnType(String columnType) { - this.targetColumnType = columnType; + Preconditions.checkState(targetColumnType.size() < 2, "The DPP contains multiple targets."); + if (targetColumnType.isEmpty()) { + targetColumnType.add(columnType); + } else { + targetColumnType.set(0, columnType); + } } - public ExprNodeDesc getPartKey() { + public List getPartKey() { return partKey; } - public void setPartKey(ExprNodeDesc partKey) { - this.partKey = partKey; + public void setPartKey(ExprNodeDesc key) { + Preconditions.checkState(partKey.size() < 2, "The DPP contains multiple targets."); + if (partKey.isEmpty()) { + partKey.add(key); + } else { + partKey.set(0, key); + } } public TableDesc getTable() { @@ -115,6 +150,19 @@ public void setTable(TableDesc table) { @Explain(displayName = "partition key expr") public String getPartKeyString() { - return partKey.getExprString(); + String[] res = new String[partKey.size()]; + for (int i = 0; i < res.length; i++) { + res[i] = partKey.get(i).getExprString(); + } + return Arrays.toString(res); + } + + @Override + public boolean isSame(OperatorDesc other) { + if (getClass().getName().equals(other.getClass().getName())) { + SparkPartitionPruningSinkDesc otherDesc = (SparkPartitionPruningSinkDesc) other; + return getTable().equals(otherDesc.getTable()); + } + return false; } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index 24c8baf..296e043 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -478,7 +478,7 @@ public void processPartitionPruningSink(GenSparkProcContext context, } desc.setPath(new Path(tmpPath, sourceId)); - desc.setTargetMapWork(targetWork); + desc.getTargetMapWorks().add(targetWork); // store table descriptor in map-targetWork if (!targetWork.getEventSourceTableDescMap().containsKey(sourceId)) { @@ -491,21 +491,23 @@ public void processPartitionPruningSink(GenSparkProcContext context, if (!targetWork.getEventSourceColumnNameMap().containsKey(sourceId)) { targetWork.getEventSourceColumnNameMap().put(sourceId, new LinkedList()); } + // in order to make the col name unique, prepend the targetId + desc.setTargetColumnName(targetId + ":" + desc.getTargetColumnName().get(0)); List columns = targetWork.getEventSourceColumnNameMap().get(sourceId); - columns.add(desc.getTargetColumnName()); + columns.add(desc.getTargetColumnName().get(0)); if (!targetWork.getEventSourceColumnTypeMap().containsKey(sourceId)) { targetWork.getEventSourceColumnTypeMap().put(sourceId, new LinkedList()); } List columnTypes = targetWork.getEventSourceColumnTypeMap().get(sourceId); - columnTypes.add(desc.getTargetColumnType()); + columnTypes.add(desc.getTargetColumnType().get(0)); // store partition key expr in map-targetWork if (!targetWork.getEventSourcePartKeyExprMap().containsKey(sourceId)) { targetWork.getEventSourcePartKeyExprMap().put(sourceId, new LinkedList()); } List keys = targetWork.getEventSourcePartKeyExprMap().get(sourceId); - keys.add(desc.getPartKey()); + keys.add(desc.getPartKey().get(0)); } public static SparkEdgeProperty getEdgeProperty(HiveConf conf, ReduceSinkOperator reduceSink, diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java index e3146cf..f8010d0 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkPartitionPruningSinkOperator.java @@ -166,7 +166,10 @@ private void flushToFile() throws IOException { try { fsout = fs.create(path, numOfRepl); out = new ObjectOutputStream(new BufferedOutputStream(fsout)); - out.writeUTF(conf.getTargetColumnName()); + out.writeInt(conf.getTargetColumnName().size()); + for (String name : conf.getTargetColumnName()) { + out.writeUTF(name); + } buffer.writeTo(out); } catch (Exception e) { try { diff --git a/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_4.q b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_4.q new file mode 100644 index 0000000..0b3384f --- /dev/null +++ b/ql/src/test/queries/clientpositive/spark_dynamic_partition_pruning_4.q @@ -0,0 +1,153 @@ +set hive.spark.dynamic.partition.pruning=true; + +-- SORT_QUERY_RESULTS + +create table part1(key string, value string) partitioned by (p string, q string); +insert into table part1 partition (p='1', q='1') values ('1','1'), ('2','2'); +insert into table part1 partition (p='1', q='2') values ('3','3'), ('4','4'); +insert into table part1 partition (p='2', q='1') values ('5','5'), ('6','6'); +insert into table part1 partition (p='2', q='2') values ('7','7'), ('8','8'); + +create table part2(key string, value string) partitioned by (p string, q string); +insert into table part2 partition (p='3', q='3') values ('a','a'), ('b','b'); +insert into table part2 partition (p='3', q='4') values ('c','c'), ('d','d'); +insert into table part2 partition (p='4', q='3') values ('e','e'), ('f','f'); +insert into table part2 partition (p='4', q='4') values ('g','g'), ('h','h'); + +-- dpp works should be combined +explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key); + +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key); + +-- verify result +set hive.spark.dynamic.partition.pruning=false; + +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key); + +set hive.spark.dynamic.partition.pruning=true; + +-- dpp works should be combined +explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key); + +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key); + +-- verify result +set hive.spark.dynamic.partition.pruning=false; + +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key); + +set hive.spark.dynamic.partition.pruning=true; + +-- target works are already combined +explain +select * from + (select part1.key, part1.value from part1 join src on part1.q=src.key) a +union all + (select part1.key, part1.value from part1 join src on part1.q=src.key); + +select * from + (select part1.key, part1.value from part1 join src on part1.q=src.key) a +union all + (select part1.key, part1.value from part1 join src on part1.q=src.key); + +-- dpp works shouldn't be combined +explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.value); + +-- dpp works shouldn't be combined +explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=upper(src.key)) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key); + +-- dpp works should be combined +explain +with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key); + +with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key); + +-- verify result +set hive.spark.dynamic.partition.pruning=false; + +with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key); + +set hive.spark.dynamic.partition.pruning=true; + +-- dpp works should be combined +explain +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key); + +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key); + +--verify result +set hive.spark.dynamic.partition.pruning=false; + +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key); + +set hive.spark.dynamic.partition.pruning=true; + +-- dpp works shouldn't be combined +explain +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.value); + +drop table part1; +drop table part2; \ No newline at end of file diff --git a/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_4.q.out b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_4.q.out new file mode 100644 index 0000000..20fa5a7 --- /dev/null +++ b/ql/src/test/results/clientpositive/spark/spark_dynamic_partition_pruning_4.q.out @@ -0,0 +1,1891 @@ +PREHOOK: query: create table part1(key string, value string) partitioned by (p string, q string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@part1 +POSTHOOK: query: create table part1(key string, value string) partitioned by (p string, q string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@part1 +PREHOOK: query: insert into table part1 partition (p='1', q='1') values ('1','1'), ('2','2') +PREHOOK: type: QUERY +PREHOOK: Output: default@part1@p=1/q=1 +POSTHOOK: query: insert into table part1 partition (p='1', q='1') values ('1','1'), ('2','2') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part1@p=1/q=1 +POSTHOOK: Lineage: part1 PARTITION(p=1,q=1).key SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part1 PARTITION(p=1,q=1).value SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into table part1 partition (p='1', q='2') values ('3','3'), ('4','4') +PREHOOK: type: QUERY +PREHOOK: Output: default@part1@p=1/q=2 +POSTHOOK: query: insert into table part1 partition (p='1', q='2') values ('3','3'), ('4','4') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part1@p=1/q=2 +POSTHOOK: Lineage: part1 PARTITION(p=1,q=2).key SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part1 PARTITION(p=1,q=2).value SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into table part1 partition (p='2', q='1') values ('5','5'), ('6','6') +PREHOOK: type: QUERY +PREHOOK: Output: default@part1@p=2/q=1 +POSTHOOK: query: insert into table part1 partition (p='2', q='1') values ('5','5'), ('6','6') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part1@p=2/q=1 +POSTHOOK: Lineage: part1 PARTITION(p=2,q=1).key SIMPLE [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part1 PARTITION(p=2,q=1).value SIMPLE [(values__tmp__table__3)values__tmp__table__3.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into table part1 partition (p='2', q='2') values ('7','7'), ('8','8') +PREHOOK: type: QUERY +PREHOOK: Output: default@part1@p=2/q=2 +POSTHOOK: query: insert into table part1 partition (p='2', q='2') values ('7','7'), ('8','8') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part1@p=2/q=2 +POSTHOOK: Lineage: part1 PARTITION(p=2,q=2).key SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part1 PARTITION(p=2,q=2).value SIMPLE [(values__tmp__table__4)values__tmp__table__4.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: create table part2(key string, value string) partitioned by (p string, q string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@part2 +POSTHOOK: query: create table part2(key string, value string) partitioned by (p string, q string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@part2 +PREHOOK: query: insert into table part2 partition (p='3', q='3') values ('a','a'), ('b','b') +PREHOOK: type: QUERY +PREHOOK: Output: default@part2@p=3/q=3 +POSTHOOK: query: insert into table part2 partition (p='3', q='3') values ('a','a'), ('b','b') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part2@p=3/q=3 +POSTHOOK: Lineage: part2 PARTITION(p=3,q=3).key SIMPLE [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part2 PARTITION(p=3,q=3).value SIMPLE [(values__tmp__table__5)values__tmp__table__5.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into table part2 partition (p='3', q='4') values ('c','c'), ('d','d') +PREHOOK: type: QUERY +PREHOOK: Output: default@part2@p=3/q=4 +POSTHOOK: query: insert into table part2 partition (p='3', q='4') values ('c','c'), ('d','d') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part2@p=3/q=4 +POSTHOOK: Lineage: part2 PARTITION(p=3,q=4).key SIMPLE [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part2 PARTITION(p=3,q=4).value SIMPLE [(values__tmp__table__6)values__tmp__table__6.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into table part2 partition (p='4', q='3') values ('e','e'), ('f','f') +PREHOOK: type: QUERY +PREHOOK: Output: default@part2@p=4/q=3 +POSTHOOK: query: insert into table part2 partition (p='4', q='3') values ('e','e'), ('f','f') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part2@p=4/q=3 +POSTHOOK: Lineage: part2 PARTITION(p=4,q=3).key SIMPLE [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part2 PARTITION(p=4,q=3).value SIMPLE [(values__tmp__table__7)values__tmp__table__7.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: insert into table part2 partition (p='4', q='4') values ('g','g'), ('h','h') +PREHOOK: type: QUERY +PREHOOK: Output: default@part2@p=4/q=4 +POSTHOOK: query: insert into table part2 partition (p='4', q='4') values ('g','g'), ('h','h') +POSTHOOK: type: QUERY +POSTHOOK: Output: default@part2@p=4/q=4 +POSTHOOK: Lineage: part2 PARTITION(p=4,q=4).key SIMPLE [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col1, type:string, comment:), ] +POSTHOOK: Lineage: part2 PARTITION(p=4,q=4).value SIMPLE [(values__tmp__table__8)values__tmp__table__8.FieldSchema(name:tmp_values_col2, type:string, comment:), ] +PREHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + outputColumnNames: _col0 + 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 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [3:p (string), 6:p (string)] + partition key expr: [p, p] + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + target works: [Map 3, Map 6] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 6 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + 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 + Map 3 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: part2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +5 5 +6 6 +7 7 +8 8 +e e +f f +g g +h h +PREHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +5 5 +6 6 +7 7 +8 8 +e e +f f +g g +h h +PREHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + outputColumnNames: _col0 + 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 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [3:p (string), 6:q (string)] + partition key expr: [p, q] + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + target works: [Map 3, Map 6] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 6 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + 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 + Map 3 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 6 + Map Operator Tree: + TableScan + alias: part2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +5 5 +6 6 +7 7 +8 8 +c c +d d +g g +h h +PREHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.q=src.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +5 5 +6 6 +7 7 +8 8 +c c +d d +g g +h h +PREHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.q=src.key) a +union all + (select part1.key, part1.value from part1 join src on part1.q=src.key) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.q=src.key) a +union all + (select part1.key, part1.value from part1 join src on part1.q=src.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + outputColumnNames: _col0 + 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 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [3:q (string)] + partition key expr: [q] + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + target works: [Map 3] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) + Reducer 5 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + 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 + Map 3 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.q=src.key) a +union all + (select part1.key, part1.value from part1 join src on part1.q=src.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: select * from + (select part1.key, part1.value from part1 join src on part1.q=src.key) a +union all + (select part1.key, part1.value from part1 join src on part1.q=src.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +3 3 +3 3 +4 4 +4 4 +7 7 +7 7 +8 8 +8 8 +PREHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.value) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=src.key) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.value) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + outputColumnNames: _col0 + 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 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [3:p (string)] + partition key expr: [p] + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + target works: [Map 3] + Map 8 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + outputColumnNames: _col0 + 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 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [6:p (string)] + partition key expr: [p] + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + target works: [Map 6] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 4), Map 6 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + 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 + Map 3 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + 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 + Map 6 + Map Operator Tree: + TableScan + alias: part2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=upper(src.key)) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +PREHOOK: type: QUERY +POSTHOOK: query: explain +select * from + (select part1.key, part1.value from part1 join src on part1.p=upper(src.key)) a +union all + (select part2.key, part2.value from part2 join src on part2.p=src.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark +#### A masked pattern was here #### + Vertices: + Map 7 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: upper(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) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: upper(_col0) (type: string) + 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 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [3:p (string)] + partition key expr: [p] + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + target works: [Map 3] + Map 8 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + outputColumnNames: _col0 + 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 + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [6:p (string)] + partition key expr: [p] + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + target works: [Map 6] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 4), Map 3 (PARTITION-LEVEL SORT, 4) + Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 4), Map 6 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: upper(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) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: upper(_col0) (type: string) + sort order: + + Map-reduce partition columns: upper(_col0) (type: string) + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Map 3 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: 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) + 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 + Map 6 + Map Operator Tree: + TableScan + alias: part2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 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 + Reducer 5 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 1100 Data size: 11686 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: explain +with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key) +PREHOOK: type: QUERY +POSTHOOK: query: explain +with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 10 <- Map 11 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 11 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 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 + 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 + Reducer 10 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [4:q (string), 8:q (string)] + partition key expr: [q, q] + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + target works: [Map 4, Map 8] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 4), Reducer 2 (PARTITION-LEVEL SORT, 4) + Reducer 7 <- Map 8 (PARTITION-LEVEL SORT, 4), Reducer 2 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: src + Statistics: Num rows: 500 Data size: 5312 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 + 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 + Map 4 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: part2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 200 Data size: 2000 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: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 440 Data size: 4400 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 + Reducer 7 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col1, _col2 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col1 (type: string), _col2 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 440 Data size: 4400 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +3 3 +4 4 +7 7 +8 8 +PREHOOK: query: with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: with top as +(select key from src order by key limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.q=top.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +3 3 +4 4 +7 7 +8 8 +PREHOOK: query: explain +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key) +PREHOOK: type: QUERY +POSTHOOK: query: explain +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 10 <- Map 11 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 11 + Map Operator Tree: + 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 + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Reducer 10 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [4:q (string), 8:q (string)] + partition key expr: [q, q] + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + target works: [Map 4, Map 8] + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [4:p (string), 8:p (string)] + partition key expr: [p, p] + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + target works: [Map 4, Map 8] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 4), Reducer 2 (PARTITION-LEVEL SORT, 4) + Reducer 7 <- Map 8 (PARTITION-LEVEL SORT, 4), Reducer 2 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + 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 + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Map 4 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: _col3 (type: string), _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: part2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: _col3 (type: string), _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col0 (type: string) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col2, _col3 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 440 Data size: 4400 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 + Reducer 7 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col2, _col3 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 440 Data size: 4400 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +7 7 +8 8 +PREHOOK: query: with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key) +PREHOOK: type: QUERY +PREHOOK: Input: default@part1 +PREHOOK: Input: default@part1@p=1/q=1 +PREHOOK: Input: default@part1@p=1/q=2 +PREHOOK: Input: default@part1@p=2/q=1 +PREHOOK: Input: default@part1@p=2/q=2 +PREHOOK: Input: default@part2 +PREHOOK: Input: default@part2@p=3/q=3 +PREHOOK: Input: default@part2@p=3/q=4 +PREHOOK: Input: default@part2@p=4/q=3 +PREHOOK: Input: default@part2@p=4/q=4 +PREHOOK: Input: default@src +#### A masked pattern was here #### +POSTHOOK: query: with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.key) +POSTHOOK: type: QUERY +POSTHOOK: Input: default@part1 +POSTHOOK: Input: default@part1@p=1/q=1 +POSTHOOK: Input: default@part1@p=1/q=2 +POSTHOOK: Input: default@part1@p=2/q=1 +POSTHOOK: Input: default@part1@p=2/q=2 +POSTHOOK: Input: default@part2 +POSTHOOK: Input: default@part2@p=3/q=3 +POSTHOOK: Input: default@part2@p=3/q=4 +POSTHOOK: Input: default@part2@p=4/q=3 +POSTHOOK: Input: default@part2@p=4/q=4 +POSTHOOK: Input: default@src +#### A masked pattern was here #### +7 7 +8 8 +PREHOOK: query: explain +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.value) +PREHOOK: type: QUERY +POSTHOOK: query: explain +with top as +(select key, value from src order by key, value limit 200) +select * from + (select part1.key, part1.value from part1 join top on part1.p=top.key and part1.q=top.key) a +union all + (select part2.key, part2.value from part2 join top on part2.p=top.key and part2.q=top.value) +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-1 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 10 <- Map 11 (PARTITION-LEVEL SORT, 1) + Reducer 12 <- Map 11 (PARTITION-LEVEL SORT, 1) +#### A masked pattern was here #### + Vertices: + Map 11 + Map Operator Tree: + 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 + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Reducer 10 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [4:q (string)] + partition key expr: [q] + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + target works: [Map 4] + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [4:p (string)] + partition key expr: [p] + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + target works: [Map 4] + Reducer 12 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col0 is not null and _col1 is not null) (type: boolean) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [8:p (string)] + partition key expr: [p] + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + target works: [Map 8] + Select Operator + expressions: _col1 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Group By Operator + keys: _col0 (type: string) + mode: hash + outputColumnNames: _col0 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Spark Partition Pruning Sink Operator + Target column: [8:q (string)] + partition key expr: [q] + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + target works: [Map 8] + + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (SORT, 1) + Reducer 3 <- Map 4 (PARTITION-LEVEL SORT, 4), Reducer 2 (PARTITION-LEVEL SORT, 4) + Reducer 6 <- Map 1 (SORT, 1) + Reducer 7 <- Map 8 (PARTITION-LEVEL SORT, 4), Reducer 6 (PARTITION-LEVEL SORT, 4) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + 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 + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + TopN Hash Memory Usage: 0.1 + Map 4 + Map Operator Tree: + TableScan + alias: part1 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col3 (type: string), _col2 (type: string) + sort order: ++ + Map-reduce partition columns: _col3 (type: string), _col2 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Map 8 + Map Operator Tree: + TableScan + alias: part2 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), p (type: string), q (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string), _col3 (type: string) + sort order: ++ + Map-reduce partition columns: _col2 (type: string), _col3 (type: string) + Statistics: Num rows: 8 Data size: 24 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: _col0 is not null (type: boolean) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col0 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col0 (type: string) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Reducer 3 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col2, _col3 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 440 Data size: 4400 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 + Reducer 6 + Reduce Operator Tree: + Select Operator + expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 200 + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (_col0 is not null and _col1 is not null) (type: boolean) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 200 Data size: 2000 Basic stats: COMPLETE Column stats: NONE + Reducer 7 + Reduce Operator Tree: + Join Operator + condition map: + Inner Join 0 to 1 + outputColumnNames: _col2, _col3 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 220 Data size: 2200 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 440 Data size: 4400 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + + Stage: Stage-0 + Fetch Operator + limit: -1 + Processor Tree: + ListSink + +PREHOOK: query: drop table part1 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@part1 +PREHOOK: Output: default@part1 +POSTHOOK: query: drop table part1 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@part1 +POSTHOOK: Output: default@part1 +PREHOOK: query: drop table part2 +PREHOOK: type: DROPTABLE +PREHOOK: Input: default@part2 +PREHOOK: Output: default@part2 +POSTHOOK: query: drop table part2 +POSTHOOK: type: DROPTABLE +POSTHOOK: Input: default@part2 +POSTHOOK: Output: default@part2