From 7943fd85e87ca89e3036e3590a31fba9ae4b5a35 Mon Sep 17 00:00:00 2001 From: Na Yang Date: Thu, 28 Aug 2014 23:29:44 -0700 Subject: [PATCH] HIVE-7870: Insert overwrite table query does not generate correct task plan [Spark Branch] --- .../test/resources/testconfiguration.properties | 22 +- .../hadoop/hive/ql/optimizer/GenMapRedUtils.java | 6 + .../hive/ql/parse/spark/GenSparkProcContext.java | 13 + .../hadoop/hive/ql/parse/spark/GenSparkUtils.java | 27 +- .../hadoop/hive/ql/parse/spark/SparkCompiler.java | 43 ++ .../queries/clientpositive/union_remove_spark_1.q | 45 ++ .../queries/clientpositive/union_remove_spark_10.q | 58 ++ .../queries/clientpositive/union_remove_spark_11.q | 58 ++ .../queries/clientpositive/union_remove_spark_15.q | 54 ++ .../queries/clientpositive/union_remove_spark_16.q | 53 ++ .../queries/clientpositive/union_remove_spark_17.q | 49 ++ .../queries/clientpositive/union_remove_spark_18.q | 53 ++ .../queries/clientpositive/union_remove_spark_19.q | 91 +++ .../queries/clientpositive/union_remove_spark_2.q | 50 ++ .../queries/clientpositive/union_remove_spark_20.q | 46 ++ .../queries/clientpositive/union_remove_spark_21.q | 46 ++ .../queries/clientpositive/union_remove_spark_24.q | 44 ++ .../queries/clientpositive/union_remove_spark_25.q | 87 +++ .../queries/clientpositive/union_remove_spark_3.q | 50 ++ .../queries/clientpositive/union_remove_spark_4.q | 46 ++ .../queries/clientpositive/union_remove_spark_5.q | 52 ++ .../queries/clientpositive/union_remove_spark_6.q | 42 ++ .../queries/clientpositive/union_remove_spark_7.q | 47 ++ .../queries/clientpositive/union_remove_spark_8.q | 52 ++ .../queries/clientpositive/union_remove_spark_9.q | 56 ++ .../results/clientpositive/spark/sample8.q.out | 30 +- .../results/clientpositive/spark/union10.q.out | 15 - .../results/clientpositive/spark/union18.q.out | 20 - .../results/clientpositive/spark/union19.q.out | 10 - .../results/clientpositive/spark/union28.q.out | 15 - .../results/clientpositive/spark/union29.q.out | 15 - .../test/results/clientpositive/spark/union3.q.out | 4 - .../results/clientpositive/spark/union30.q.out | 20 - .../results/clientpositive/spark/union33.q.out | 20 - .../test/results/clientpositive/spark/union4.q.out | 10 - .../test/results/clientpositive/spark/union6.q.out | 10 - .../spark/union_remove_spark_1.q.out | 247 ++++++++ .../spark/union_remove_spark_10.q.out | 303 ++++++++++ .../spark/union_remove_spark_11.q.out | 285 +++++++++ .../spark/union_remove_spark_15.q.out | 278 +++++++++ .../spark/union_remove_spark_16.q.out | 306 ++++++++++ .../spark/union_remove_spark_17.q.out | 236 ++++++++ .../spark/union_remove_spark_18.q.out | 318 ++++++++++ .../spark/union_remove_spark_19.q.out | 603 +++++++++++++++++++ .../spark/union_remove_spark_2.q.out | 256 ++++++++ .../spark/union_remove_spark_20.q.out | 249 ++++++++ .../spark/union_remove_spark_21.q.out | 247 ++++++++ .../spark/union_remove_spark_24.q.out | 245 ++++++++ .../spark/union_remove_spark_25.q.out | 666 +++++++++++++++++++++ .../spark/union_remove_spark_3.q.out | 235 ++++++++ .../spark/union_remove_spark_4.q.out | 297 +++++++++ .../spark/union_remove_spark_5.q.out | 308 ++++++++++ .../spark/union_remove_spark_6.q.out | 269 +++++++++ .../spark/union_remove_spark_7.q.out | 251 ++++++++ .../spark/union_remove_spark_8.q.out | 260 ++++++++ .../spark/union_remove_spark_9.q.out | 299 +++++++++ .../clientpositive/union_remove_spark_1.q.out | 238 ++++++++ .../clientpositive/union_remove_spark_10.q.out | 315 ++++++++++ .../clientpositive/union_remove_spark_11.q.out | 294 +++++++++ .../clientpositive/union_remove_spark_15.q.out | 269 +++++++++ .../clientpositive/union_remove_spark_16.q.out | 297 +++++++++ .../clientpositive/union_remove_spark_17.q.out | 233 +++++++ .../clientpositive/union_remove_spark_18.q.out | 309 ++++++++++ .../clientpositive/union_remove_spark_19.q.out | 576 ++++++++++++++++++ .../clientpositive/union_remove_spark_2.q.out | 252 ++++++++ .../clientpositive/union_remove_spark_20.q.out | 240 ++++++++ .../clientpositive/union_remove_spark_21.q.out | 238 ++++++++ .../clientpositive/union_remove_spark_24.q.out | 236 ++++++++ .../clientpositive/union_remove_spark_25.q.out | 639 ++++++++++++++++++++ .../clientpositive/union_remove_spark_3.q.out | 238 ++++++++ .../clientpositive/union_remove_spark_4.q.out | 282 +++++++++ .../clientpositive/union_remove_spark_5.q.out | 298 +++++++++ .../clientpositive/union_remove_spark_6.q.out | 292 +++++++++ .../clientpositive/union_remove_spark_7.q.out | 242 ++++++++ .../clientpositive/union_remove_spark_8.q.out | 256 ++++++++ .../clientpositive/union_remove_spark_9.q.out | 269 +++++++++ 76 files changed, 13373 insertions(+), 157 deletions(-) create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_1.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_10.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_11.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_15.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_16.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_17.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_18.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_19.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_2.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_20.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_21.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_24.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_25.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_3.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_4.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_5.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_6.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_7.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_8.q create mode 100644 ql/src/test/queries/clientpositive/union_remove_spark_9.q create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_1.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_10.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_11.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_15.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_16.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_17.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_18.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_19.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_2.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_20.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_21.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_24.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_25.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_3.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_4.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_5.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_6.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_7.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_8.q.out create mode 100644 ql/src/test/results/clientpositive/spark/union_remove_spark_9.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_1.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_10.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_11.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_15.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_16.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_17.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_18.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_19.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_2.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_20.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_21.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_24.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_25.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_3.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_4.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_5.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_6.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_7.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_8.q.out create mode 100644 ql/src/test/results/clientpositive/union_remove_spark_9.q.out diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index 6393671..37945a5 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -452,4 +452,24 @@ spark.query.files=alter_merge_orc.q \ union_remove_6.q \ union_remove_7.q \ union_remove_8.q \ - union_remove_9.q + union_remove_9.q \ + union_remove_spark_1.q \ + union_remove_spark_10.q \ + union_remove_spark_11.q \ + union_remove_spark_15.q \ + union_remove_spark_16.q \ + union_remove_spark_17.q \ + union_remove_spark_18.q \ + union_remove_spark_19.q \ + union_remove_spark_2.q \ + union_remove_spark_20.q \ + union_remove_spark_21.q \ + union_remove_spark_24.q \ + union_remove_spark_25.q \ + union_remove_spark_3.q \ + union_remove_spark_4.q \ + union_remove_spark_5.q \ + union_remove_spark_6.q \ + union_remove_spark_7.q \ + union_remove_spark_8.q \ + union_remove_spark_9.q diff --git ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java index 9c808d4..daaa377 100644 --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMapRedUtils.java @@ -1744,6 +1744,12 @@ public static Path createMoveTask(Task currTask, boolean } } else { fileSinkDesc.setDirName(tmpDir); + if (fileSinkDesc.getLinkedFileSinkDesc() != null) { + for (FileSinkDesc fsConf:fileSinkDesc.getLinkedFileSinkDesc()) { + fsConf.setParentDir(tmpDir); + fsConf.setDirName(tmpDir); + } + } } } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java index 5ddc16d..4ab7300 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java @@ -132,6 +132,10 @@ // remember which reducesinks we've already connected public final Set connectedReduceSinks; + //If many fileSinkDescs are linked to each other, it is a good idea to keep track of + // tasks for first fileSinkDesc. others can use it + private Map> linkedFileDescTasks; + @SuppressWarnings("unchecked") public GenSparkProcContext(HiveConf conf, ParseContext parseContext, List> moveTask, List> rootTasks, @@ -166,4 +170,13 @@ public GenSparkProcContext(HiveConf conf, ParseContext parseContext, rootTasks.add(currentTask); } + public Map> getLinkedFileDescTasks() { + return linkedFileDescTasks; + } + + public void setLinkedFileDescTasks( + Map> linkedFileDescTasks) { + this.linkedFileDescTasks = linkedFileDescTasks; + } + } diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java index 379a39c..fed0b74 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.parse.spark; +import java.io.Serializable; import java.util.ArrayList; import java.util.Deque; import java.util.HashMap; @@ -29,6 +30,7 @@ import java.util.Set; import com.google.common.base.Strings; + import org.apache.hadoop.fs.Path; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -273,7 +275,15 @@ public void processFileSink(GenSparkProcContext context, FileSinkOperator fileSi boolean isInsertTable = // is INSERT OVERWRITE TABLE GenMapRedUtils.isInsertInto(parseContext, fileSink); HiveConf hconf = parseContext.getConf(); - + // If this file sink desc has been processed due to a linked file sink desc, + // use that task + Map> fileSinkDescs = context.getLinkedFileDescTasks(); + if (fileSinkDescs != null) { + Task task = fileSinkDescs.get(fileSink.getConf()); + if (task != null) { + return; + } + } boolean chDir = GenMapRedUtils.isMergeRequired(context.moveTask, hconf, fileSink, context.currentTask, isInsertTable); @@ -288,7 +298,20 @@ public void processFileSink(GenSparkProcContext context, FileSinkOperator fileSi context.dependencyTask, context.moveTask, hconf, context.currentTask); } - + FileSinkDesc fileSinkDesc = fileSink.getConf(); + if (fileSinkDesc.getLinkedFileSinkDesc() != null) { + Map> linkedFileDescTasks = + context.getLinkedFileDescTasks(); + if (linkedFileDescTasks == null) { + linkedFileDescTasks = new HashMap>(); + context.setLinkedFileDescTasks(linkedFileDescTasks); + } + if (context.currentTask != null) { + for (FileSinkDesc fileDesc : fileSinkDesc.getLinkedFileSinkDesc()) { + linkedFileDescTasks.put(fileDesc, context.currentTask); + } + } + } FetchTask fetchTask = parseContext.getFetchTask(); if (fetchTask != null && context.currentTask.getNumChild() == 0) { if (fetchTask.isFetchFrom(fileSink.getConf())) { diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java index 76fc290..74cc394 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.Deque; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedList; import java.util.List; @@ -30,6 +31,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.Context; import org.apache.hadoop.hive.ql.exec.ConditionalTask; @@ -64,6 +66,7 @@ import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.parse.TaskCompiler; import org.apache.hadoop.hive.ql.plan.BaseWork; +import org.apache.hadoop.hive.ql.plan.FileSinkDesc; import org.apache.hadoop.hive.ql.plan.MapWork; import org.apache.hadoop.hive.ql.plan.MoveWork; import org.apache.hadoop.hive.ql.plan.OperatorDesc; @@ -183,6 +186,46 @@ public Object process(Node n, Stack s, GenSparkUtils.getUtils().removeUnionOperators(conf, procCtx, w); } + // re-construct the linkedfilesinkdesc for all fileSinks + Map> linkedFileSinkDescMap = new HashMap>(); + for (FileSinkOperator fileSink: procCtx.fileSinkSet) { + FileSinkDesc desc = fileSink.getConf(); + Path path = desc.getDirName(); + if (linkedFileSinkDescMap.containsKey(path)) { + Set descSet = linkedFileSinkDescMap.get(path); + List descList = desc.getLinkedFileSinkDesc(); + if (descList != null) { + for (FileSinkDesc fsDesc : descList) { + descSet.add(fsDesc); + } + } + } else { + Set descSet = new HashSet(); + List descList = desc.getLinkedFileSinkDesc(); + if (descList != null) { + for (FileSinkDesc fsDesc : descList) { + descSet.add(fsDesc); + } + linkedFileSinkDescMap.put(path, descSet); + } + } + } + + for (FileSinkOperator fileSink: procCtx.fileSinkSet) { + FileSinkDesc desc = fileSink.getConf(); + Path path = desc.getDirName(); + Set descSet = linkedFileSinkDescMap.get(path); + List linked = null; + if (descSet != null) { + linked = new ArrayList(); + for (FileSinkDesc fsDesc : descSet) { + linked.add(fsDesc); + } + } + if (linked != null) { + desc.setLinkedFileSinkDesc(linked); + } + } // finally make sure the file sink operators are set up right for (FileSinkOperator fileSink: procCtx.fileSinkSet) { GenSparkUtils.getUtils().processFileSink(procCtx, fileSink); diff --git ql/src/test/queries/clientpositive/union_remove_spark_1.q ql/src/test/queries/clientpositive/union_remove_spark_1.q new file mode 100644 index 0000000..0db1743 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_1.q @@ -0,0 +1,45 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_10.q ql/src/test/queries/clientpositive/union_remove_spark_10.q new file mode 100644 index 0000000..1e9c201 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_10.q @@ -0,0 +1,58 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=true; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where one of the sub-queries requires a map-reduce +-- job), followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The outer union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as rcfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_11.q ql/src/test/queries/clientpositive/union_remove_spark_11.q new file mode 100644 index 0000000..7052c69 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_11.q @@ -0,0 +1,58 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=true; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where also contains map only sub-queries), +-- followed by select star and a file sink. +-- There is no need for the union optimization, since the whole query can be performed +-- in a single map-only job +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as rcfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_15.q ql/src/test/queries/clientpositive/union_remove_spark_15.q new file mode 100644 index 0000000..72ced75 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_15.q @@ -0,0 +1,54 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This tests demonstrates that this optimization works in the presence of dynamic partitions. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '1' order by key, values; +select * from outputTbl1 where ds = '2' order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_16.q ql/src/test/queries/clientpositive/union_remove_spark_16.q new file mode 100644 index 0000000..72e6cb1 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_16.q @@ -0,0 +1,53 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=true; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- This test demonstrates that this optimization works in the presence of dynamic partitions. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile ; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a; + +desc formatted outputTbl1; +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '1' order by key, values; +select * from outputTbl1 where ds = '2' order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_17.q ql/src/test/queries/clientpositive/union_remove_spark_17.q new file mode 100644 index 0000000..eb17c02 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_17.q @@ -0,0 +1,49 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need for this optimization, since the query is a map-only query. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a; + +desc formatted outputTbl1; +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '1' order by key, values; +select * from outputTbl1 where ds = '2' order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_18.q ql/src/test/queries/clientpositive/union_remove_spark_18.q new file mode 100644 index 0000000..92ff57e --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_18.q @@ -0,0 +1,53 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This test demonstrates that the optimization works with dynamic partitions irrespective of the +-- file format of the output file +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, ds string) stored as textfile; +create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a; + +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a; + +desc formatted outputTbl1; + +show partitions outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 where ds = '11' order by key, values; +select * from outputTbl1 where ds = '18' order by key, values; +select * from outputTbl1 where ds is not null order by key, values, ds; diff --git ql/src/test/queries/clientpositive/union_remove_spark_19.q ql/src/test/queries/clientpositive/union_remove_spark_19.q new file mode 100644 index 0000000..36a1e9b --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_19.q @@ -0,0 +1,91 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +select * from outputTbl1; + +-- filter should be fine +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7; + +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7; + +select * from outputTbl1; + +-- filters and sub-queries should be fine +explain +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7; + +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_2.q ql/src/test/queries/clientpositive/union_remove_spark_2.q new file mode 100644 index 0000000..4ac04d0 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_2.q @@ -0,0 +1,50 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_20.q ql/src/test/queries/clientpositive/union_remove_spark_20.q new file mode 100644 index 0000000..1c59ef2 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_20.q @@ -0,0 +1,46 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, the order of the columns in the select list is different. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(values bigint, key string) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_21.q ql/src/test/queries/clientpositive/union_remove_spark_21.q new file mode 100644 index 0000000..1ea138e --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_21.q @@ -0,0 +1,46 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, all the columns are not selected. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_24.q ql/src/test/queries/clientpositive/union_remove_spark_24.q new file mode 100644 index 0000000..6a1f252 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_24.q @@ -0,0 +1,44 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- One sub-query has a double and the other sub-query has a bigint. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key double, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a; + +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_25.q ql/src/test/queries/clientpositive/union_remove_spark_25.q new file mode 100644 index 0000000..5d2ce95 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_25.q @@ -0,0 +1,87 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile; +create table outputTbl2(key string, values bigint) partitioned by (ds string) stored as textfile; +create table outputTbl3(key string, values bigint) partitioned by (ds string,hr string) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1 partition(ds='2004'); + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; + +explain +insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a; + +insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a; + +show partitions outputTbl2; +desc formatted outputTbl2 partition(ds='2008-04-08'); + +explain insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a; + +insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a; + +show partitions outputTbl3; +desc formatted outputTbl3 partition(ds='2008-04-08', hr='11'); \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_3.q ql/src/test/queries/clientpositive/union_remove_spark_3.q new file mode 100644 index 0000000..f9b8a9e --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_3.q @@ -0,0 +1,50 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->remove->filesink optimization +-- Union of 3 subqueries is performed (all of which are map-only queries) +-- followed by select star and a file sink. +-- There is no need for any optimization, since the whole query can be processed in +-- a single map-only job +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_4.q ql/src/test/queries/clientpositive/union_remove_spark_4.q new file mode 100644 index 0000000..44b31d6 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_4.q @@ -0,0 +1,46 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=true; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set mapred.input.dir.recursive=true; +set hive.merge.smallfiles.avgsize=1; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; diff --git ql/src/test/queries/clientpositive/union_remove_spark_5.q ql/src/test/queries/clientpositive/union_remove_spark_5.q new file mode 100644 index 0000000..d6a3815 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_5.q @@ -0,0 +1,52 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=true; +set hive.merge.mapfiles=true; +set hive.merge.mapredfiles=true; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_6.q ql/src/test/queries/clientpositive/union_remove_spark_6.q new file mode 100644 index 0000000..43512a0 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_6.q @@ -0,0 +1,42 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as textfile; +create table outputTbl2(key string, values bigint) stored as textfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select *; + +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select *; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; +select * from outputTbl2 order by key, values;; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_7.q ql/src/test/queries/clientpositive/union_remove_spark_7.q new file mode 100644 index 0000000..7b50b0c --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_7.q @@ -0,0 +1,47 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as rcfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_8.q ql/src/test/queries/clientpositive/union_remove_spark_8.q new file mode 100644 index 0000000..618c849 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_8.q @@ -0,0 +1,52 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=false; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as rcfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; \ No newline at end of file diff --git ql/src/test/queries/clientpositive/union_remove_spark_9.q ql/src/test/queries/clientpositive/union_remove_spark_9.q new file mode 100644 index 0000000..c49ec45 --- /dev/null +++ ql/src/test/queries/clientpositive/union_remove_spark_9.q @@ -0,0 +1,56 @@ +set hive.stats.autogather=false; +set hive.optimize.union.remove=true; +set hive.mapred.supports.subdirectories=true; + +set hive.merge.sparkfiles=true; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set hive.merge.smallfiles.avgsize=1; +set mapred.input.dir.recursive=true; + +-- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which contains a union and is map-only), +-- and the other one is a map-reduce query followed by select star and a file sink. +-- There is no need for the outer union. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile; +create table outputTbl1(key string, values bigint) stored as rcfile; + +load data local inpath '../../data/files/T1.txt' into table inputTbl1; + +explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b; + +desc formatted outputTbl1; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +select * from outputTbl1 order by key, values; \ No newline at end of file diff --git ql/src/test/results/clientpositive/spark/sample8.q.out ql/src/test/results/clientpositive/spark/sample8.q.out index c7e333b..0527895 100644 --- ql/src/test/results/clientpositive/spark/sample8.q.out +++ ql/src/test/results/clientpositive/spark/sample8.q.out @@ -124,7 +124,7 @@ STAGE PLANS: ds 2008-04-08 hr 11 properties: - COLUMN_STATS_ACCURATE false + COLUMN_STATS_ACCURATE true bucket_count -1 columns key,value columns.comments defaultdefault @@ -132,10 +132,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpart numFiles 1 - numRows -1 + numRows 0 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 0 serialization.ddl struct srcpart { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -170,7 +170,7 @@ STAGE PLANS: ds 2008-04-08 hr 12 properties: - COLUMN_STATS_ACCURATE false + COLUMN_STATS_ACCURATE true bucket_count -1 columns key,value columns.comments defaultdefault @@ -178,10 +178,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpart numFiles 1 - numRows -1 + numRows 0 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 0 serialization.ddl struct srcpart { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -216,7 +216,7 @@ STAGE PLANS: ds 2008-04-09 hr 11 properties: - COLUMN_STATS_ACCURATE false + COLUMN_STATS_ACCURATE true bucket_count -1 columns key,value columns.comments defaultdefault @@ -224,10 +224,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpart numFiles 1 - numRows -1 + numRows 0 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 0 serialization.ddl struct srcpart { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -262,7 +262,7 @@ STAGE PLANS: ds 2008-04-09 hr 12 properties: - COLUMN_STATS_ACCURATE false + COLUMN_STATS_ACCURATE true bucket_count -1 columns key,value columns.comments defaultdefault @@ -270,10 +270,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpart numFiles 1 - numRows -1 + numRows 0 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 0 serialization.ddl struct srcpart { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -334,7 +334,7 @@ STAGE PLANS: ds 2008-04-08 hr 11 properties: - COLUMN_STATS_ACCURATE false + COLUMN_STATS_ACCURATE true bucket_count -1 columns key,value columns.comments defaultdefault @@ -342,10 +342,10 @@ STAGE PLANS: #### A masked pattern was here #### name default.srcpart numFiles 1 - numRows -1 + numRows 0 partition_columns ds/hr partition_columns.types string:string - rawDataSize -1 + rawDataSize 0 serialization.ddl struct srcpart { string key, string value} serialization.format 1 serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git ql/src/test/results/clientpositive/spark/union10.q.out ql/src/test/results/clientpositive/spark/union10.q.out index 20c681e..2b67840 100644 --- ql/src/test/results/clientpositive/spark/union10.q.out +++ ql/src/test/results/clientpositive/spark/union10.q.out @@ -31,9 +31,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -169,15 +166,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - - Stage: Stage-6 - Stats-Aggr Operator - PREHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 UNION ALL @@ -188,9 +176,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmptable [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 UNION ALL diff --git ql/src/test/results/clientpositive/spark/union18.q.out ql/src/test/results/clientpositive/spark/union18.q.out index 3f37a0a..658efb7 100644 --- ql/src/test/results/clientpositive/spark/union18.q.out +++ ql/src/test/results/clientpositive/spark/union18.q.out @@ -37,12 +37,8 @@ STAGE DEPENDENCIES: Stage-3 depends on stages: Stage-2 Stage-0 depends on stages: Stage-3 Stage-4 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-0 - Stage-8 depends on stages: Stage-0 Stage-1 depends on stages: Stage-3 Stage-5 depends on stages: Stage-1 - Stage-7 depends on stages: Stage-1 - Stage-9 depends on stages: Stage-1 STAGE PLANS: Stage: Stage-2 @@ -143,12 +139,6 @@ STAGE PLANS: Stage: Stage-4 Stats-Aggr Operator - Stage: Stage-6 - Stats-Aggr Operator - - Stage: Stage-8 - Stats-Aggr Operator - Stage: Stage-1 Move Operator tables: @@ -162,12 +152,6 @@ STAGE PLANS: Stage: Stage-5 Stats-Aggr Operator - Stage: Stage-7 - Stats-Aggr Operator - - Stage: Stage-9 - Stats-Aggr Operator - PREHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL select s2.key as key, s2.value as value from src s2) unionsrc @@ -179,10 +163,6 @@ PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL select s2.key as key, s2.value as value from src s2) unionsrc diff --git ql/src/test/results/clientpositive/spark/union19.q.out ql/src/test/results/clientpositive/spark/union19.q.out index 6922fcd..91dc048 100644 --- ql/src/test/results/clientpositive/spark/union19.q.out +++ ql/src/test/results/clientpositive/spark/union19.q.out @@ -39,8 +39,6 @@ STAGE DEPENDENCIES: Stage-4 depends on stages: Stage-0 Stage-1 depends on stages: Stage-3 Stage-5 depends on stages: Stage-1 - Stage-6 depends on stages: Stage-1 - Stage-7 depends on stages: Stage-1 STAGE PLANS: Stage: Stage-2 @@ -181,12 +179,6 @@ STAGE PLANS: Stage: Stage-5 Stats-Aggr Operator - Stage: Stage-6 - Stats-Aggr Operator - - Stage: Stage-7 - Stats-Aggr Operator - PREHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL select s2.key as key, s2.value as value from src s2) unionsrc @@ -198,8 +190,6 @@ PREHOOK: Output: default@dest1 PREHOOK: Output: default@dest2 [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL select s2.key as key, s2.value as value from src s2) unionsrc diff --git ql/src/test/results/clientpositive/spark/union28.q.out ql/src/test/results/clientpositive/spark/union28.q.out index 8bd5218..c55aafb 100644 --- ql/src/test/results/clientpositive/spark/union28.q.out +++ ql/src/test/results/clientpositive/spark/union28.q.out @@ -37,9 +37,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -174,15 +171,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - - Stage: Stage-6 - Stats-Aggr Operator - PREHOOK: query: insert overwrite table union_subq_union select * from ( select key, value from src @@ -198,9 +186,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_subq_union [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_subq_union select * from ( select key, value from src diff --git ql/src/test/results/clientpositive/spark/union29.q.out ql/src/test/results/clientpositive/spark/union29.q.out index b9546ef..93c94d5 100644 --- ql/src/test/results/clientpositive/spark/union29.q.out +++ ql/src/test/results/clientpositive/spark/union29.q.out @@ -37,9 +37,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -124,15 +121,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - - Stage: Stage-6 - Stats-Aggr Operator - PREHOOK: query: insert overwrite table union_subq_union select * from ( select key, value from src @@ -148,9 +136,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_subq_union [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_subq_union select * from ( select key, value from src diff --git ql/src/test/results/clientpositive/spark/union3.q.out ql/src/test/results/clientpositive/spark/union3.q.out index 3ae6536..d238a71 100644 --- ql/src/test/results/clientpositive/spark/union3.q.out +++ ql/src/test/results/clientpositive/spark/union3.q.out @@ -239,10 +239,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_out [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_out SELECT * FROM ( diff --git ql/src/test/results/clientpositive/spark/union30.q.out ql/src/test/results/clientpositive/spark/union30.q.out index 12717a1..b1014b6 100644 --- ql/src/test/results/clientpositive/spark/union30.q.out +++ ql/src/test/results/clientpositive/spark/union30.q.out @@ -51,10 +51,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 - Stage-6 depends on stages: Stage-0 - Stage-7 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -215,18 +211,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - - Stage: Stage-6 - Stats-Aggr Operator - - Stage: Stage-7 - Stats-Aggr Operator - PREHOOK: query: insert overwrite table union_subq_union select * from ( @@ -249,10 +233,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@union_subq_union [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table union_subq_union select * from ( diff --git ql/src/test/results/clientpositive/spark/union33.q.out ql/src/test/results/clientpositive/spark/union33.q.out index b89757f..7eadb04 100644 --- ql/src/test/results/clientpositive/spark/union33.q.out +++ ql/src/test/results/clientpositive/spark/union33.q.out @@ -37,8 +37,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -142,12 +140,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - PREHOOK: query: INSERT OVERWRITE TABLE test_src SELECT key, value FROM ( SELECT key, value FROM src @@ -160,8 +152,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE test_src SELECT key, value FROM ( SELECT key, value FROM src @@ -207,8 +197,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -312,12 +300,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - PREHOOK: query: INSERT OVERWRITE TABLE test_src SELECT key, value FROM ( SELECT key, COUNT(*) AS value FROM src @@ -330,8 +312,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@test_src [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: INSERT OVERWRITE TABLE test_src SELECT key, value FROM ( SELECT key, COUNT(*) AS value FROM src diff --git ql/src/test/results/clientpositive/spark/union4.q.out ql/src/test/results/clientpositive/spark/union4.q.out index 6341cd9..fd2abc8 100644 --- ql/src/test/results/clientpositive/spark/union4.q.out +++ ql/src/test/results/clientpositive/spark/union4.q.out @@ -29,8 +29,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -130,12 +128,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - PREHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 UNION ALL @@ -144,8 +136,6 @@ PREHOOK: type: QUERY PREHOOK: Input: default@src PREHOOK: Output: default@tmptable [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, count(1) as value from src s1 UNION ALL diff --git ql/src/test/results/clientpositive/spark/union6.q.out ql/src/test/results/clientpositive/spark/union6.q.out index 263d9f4..5bbb681 100644 --- ql/src/test/results/clientpositive/spark/union6.q.out +++ ql/src/test/results/clientpositive/spark/union6.q.out @@ -29,8 +29,6 @@ STAGE DEPENDENCIES: Stage-2 depends on stages: Stage-1 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-4 depends on stages: Stage-0 - Stage-5 depends on stages: Stage-0 STAGE PLANS: Stage: Stage-1 @@ -111,12 +109,6 @@ STAGE PLANS: Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-4 - Stats-Aggr Operator - - Stage: Stage-5 - Stats-Aggr Operator - PREHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL @@ -126,8 +118,6 @@ PREHOOK: Input: default@src PREHOOK: Input: default@src1 PREHOOK: Output: default@tmptable [Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to -[Error 30017]: Skipping stats aggregation by error org.apache.hadoop.hive.ql.metadata.HiveException: [Error 30015]: Stats aggregator of type counter cannot be connected to POSTHOOK: query: insert overwrite table tmptable select unionsrc.key, unionsrc.value FROM (select 'tst1' as key, cast(count(1) as string) as value from src s1 UNION ALL diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_1.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_1.q.out new file mode 100644 index 0000000..c07fbb1 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_1.q.out @@ -0,0 +1,247 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_10.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_10.q.out new file mode 100644 index 0000000..08f1bf5 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_10.q.out @@ -0,0 +1,303 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where one of the sub-queries requires a map-reduce +-- job), followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The outer union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where one of the sub-queries requires a map-reduce +-- job), followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The outer union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 + Stage-4 + Stage-2 depends on stages: Stage-4, Stage-3, Stage-6 + Stage-0 depends on stages: Stage-2 + Stage-3 + Stage-5 + Stage-6 depends on stages: Stage-5 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Union 3 <- Map 5 (NONE), Reducer 2 (NONE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Map 5 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Union 3 + Vertex: Union 3 + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-4 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-6 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 271 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_11.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_11.q.out new file mode 100644 index 0000000..a5b459c --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_11.q.out @@ -0,0 +1,285 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where also contains map only sub-queries), +-- followed by select star and a file sink. +-- There is no need for the union optimization, since the whole query can be performed +-- in a single map-only job +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where also contains map only sub-queries), +-- followed by select star and a file sink. +-- There is no need for the union optimization, since the whole query can be performed +-- in a single map-only job +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 + Stage-4 + Stage-2 depends on stages: Stage-4, Stage-3, Stage-6 + Stage-0 depends on stages: Stage-2 + Stage-3 + Stage-5 + Stage-6 depends on stages: Stage-5 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Union 2 <- Map 1 (NONE), Map 3 (NONE), Map 4 (NONE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 3 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Union 2 + Vertex: Union 2 + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-4 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-6 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 273 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 2 +1 3 +2 1 +2 2 +2 3 +3 1 +3 2 +3 3 +7 1 +7 2 +7 3 +8 1 +8 1 +8 2 +8 2 +8 3 +8 3 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_15.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_15.q.out new file mode 100644 index 0000000..463ae63 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_15.q.out @@ -0,0 +1,278 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This tests demonstrates that this optimization works in the presence of dynamic partitions. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This tests demonstrates that this optimization works in the presence of dynamic partitions. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=1 +POSTHOOK: Output: default@outputtbl1@ds=2 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=1 +ds=2 +PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +1 1 1 +2 1 1 +3 1 1 +7 1 1 +8 2 1 +PREHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +1 1 2 +2 1 2 +3 1 2 +7 1 2 +8 2 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_16.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_16.q.out new file mode 100644 index 0000000..bacb52f --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_16.q.out @@ -0,0 +1,306 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- This test demonstrates that this optimization works in the presence of dynamic partitions. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- This test demonstrates that this optimization works in the presence of dynamic partitions. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 + Stage-4 + Stage-2 depends on stages: Stage-4, Stage-3, Stage-6 + Stage-0 depends on stages: Stage-2 + Stage-3 + Stage-5 + Stage-6 depends on stages: Stage-5 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-4 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-6 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=1 +POSTHOOK: Output: default@outputtbl1@ds=2 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=1 +ds=2 +PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +1 1 1 +2 1 1 +3 1 1 +7 1 1 +8 2 1 +PREHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +1 1 2 +2 1 2 +3 1 2 +7 1 2 +8 2 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_17.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_17.q.out new file mode 100644 index 0000000..9af6da8 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_17.q.out @@ -0,0 +1,236 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need for this optimization, since the query is a map-only query. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need for this optimization, since the query is a map-only query. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Union 2 (GROUP SORT) + Union 2 <- Map 1 (NONE), Map 4 (NONE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 2 (type: int), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + value expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: string) + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 1 (type: int), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + value expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: string) + Reducer 3 + Reduce Operator Tree: + Extract + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Union 2 + Vertex: Union 2 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=1 +POSTHOOK: Output: default@outputtbl1@ds=2 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).values EXPRESSION [] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).values EXPRESSION [] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=1 +ds=2 +PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +1 1 1 +2 1 1 +3 1 1 +7 1 1 +8 1 1 +8 1 1 +PREHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +1 2 2 +2 2 2 +3 2 2 +7 2 2 +8 2 2 +8 2 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_18.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_18.q.out new file mode 100644 index 0000000..fba6634 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_18.q.out @@ -0,0 +1,318 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This test demonstrates that the optimization works with dynamic partitions irrespective of the +-- file format of the output file +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This test demonstrates that the optimization works with dynamic partitions irrespective of the +-- file format of the output file +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), ds (type: string) + outputColumnNames: key, ds + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string), ds (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col2 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), ds (type: string) + outputColumnNames: key, ds + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string), ds (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col2 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: bigint), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: bigint), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=11 +POSTHOOK: Output: default@outputtbl1@ds=12 +POSTHOOK: Output: default@outputtbl1@ds=13 +POSTHOOK: Output: default@outputtbl1@ds=17 +POSTHOOK: Output: default@outputtbl1@ds=18 +POSTHOOK: Output: default@outputtbl1@ds=28 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=11).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=11).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=12).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=12).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=13).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=13).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=17).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=17).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=18).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=18).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=28).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=28).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=11 +ds=12 +ds=13 +ds=17 +ds=18 +ds=28 +PREHOOK: query: select * from outputTbl1 where ds = '11' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=11 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '11' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=11 +#### A masked pattern was here #### +1 1 11 +1 1 11 +PREHOOK: query: select * from outputTbl1 where ds = '18' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=18 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '18' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=18 +#### A masked pattern was here #### +8 1 18 +8 1 18 +PREHOOK: query: select * from outputTbl1 where ds is not null order by key, values, ds +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=11 +PREHOOK: Input: default@outputtbl1@ds=12 +PREHOOK: Input: default@outputtbl1@ds=13 +PREHOOK: Input: default@outputtbl1@ds=17 +PREHOOK: Input: default@outputtbl1@ds=18 +PREHOOK: Input: default@outputtbl1@ds=28 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds is not null order by key, values, ds +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=11 +POSTHOOK: Input: default@outputtbl1@ds=12 +POSTHOOK: Input: default@outputtbl1@ds=13 +POSTHOOK: Input: default@outputtbl1@ds=17 +POSTHOOK: Input: default@outputtbl1@ds=18 +POSTHOOK: Input: default@outputtbl1@ds=28 +#### A masked pattern was here #### +1 1 11 +1 1 11 +2 1 12 +2 1 12 +3 1 13 +3 1 13 +7 1 17 +7 1 17 +8 1 18 +8 1 18 +8 1 28 +8 1 28 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_19.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_19.q.out new file mode 100644 index 0000000..8dd4057 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_19.q.out @@ -0,0 +1,603 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +7 1 +2 1 +8 2 +3 1 +1 1 +7 1 +2 1 +8 2 +3 1 +1 1 +PREHOOK: query: -- filter should be fine +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +PREHOOK: type: QUERY +POSTHOOK: query: -- filter should be fine +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 30 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = 7) (type: boolean) + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '7' (type: string) + outputColumnNames: key + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 15 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: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 30 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = 7) (type: boolean) + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '7' (type: string) + outputColumnNames: key + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 15 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: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +7 1 +7 1 +PREHOOK: query: -- filters and sub-queries should be fine +explain +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +PREHOOK: type: QUERY +POSTHOOK: query: -- filters and sub-queries should be fine +explain +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: (_col0 + _col0) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (_col0 >= 7.0) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: (_col0 + _col0) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (_col0 >= 7.0) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +14.0 1 +14.0 1 +16.0 2 +16.0 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_2.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_2.q.out new file mode 100644 index 0000000..268a8d9 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_2.q.out @@ -0,0 +1,256 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Map 2 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 68 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_20.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_20.q.out new file mode 100644 index 0000000..68de623 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_20.q.out @@ -0,0 +1,249 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, the order of the columns in the select list is different. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, the order of the columns in the select list is different. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(values bigint, key string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(values bigint, key string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +values bigint +key string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +1 2 +1 3 +1 3 +1 7 +1 7 +2 8 +2 8 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_21.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_21.q.out new file mode 100644 index 0000000..384f191 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_21.q.out @@ -0,0 +1,247 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, all the columns are not selected. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, all the columns are not selected. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 20 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 +1 +2 +2 +3 +3 +7 +7 +8 +8 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_24.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_24.q.out new file mode 100644 index 0000000..0494612 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_24.q.out @@ -0,0 +1,245 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- One sub-query has a double and the other sub-query has a bigint. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- One sub-query has a double and the other sub-query has a bigint. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key double, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key double, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: UDFToDouble(UDFToLong(_col0)) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: UDFToDouble(_col0) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key double +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 60 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1.0 1 +1.0 1 +2.0 1 +2.0 1 +3.0 1 +3.0 1 +7.0 1 +7.0 1 +8.0 2 +8.0 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_25.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_25.q.out new file mode 100644 index 0000000..c58c90a --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_25.q.out @@ -0,0 +1,666 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: create table outputTbl2(key string, values bigint) partitioned by (ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl2 +POSTHOOK: query: create table outputTbl2(key string, values bigint) partitioned by (ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl2 +PREHOOK: query: create table outputTbl3(key string, values bigint) partitioned by (ds string,hr string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl3 +POSTHOOK: query: create table outputTbl3(key string, values bigint) partitioned by (ds string,hr string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl3 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2004 + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1@ds=2004 +POSTHOOK: query: insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=2004 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2004).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2004).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 partition(ds='2004') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 partition(ds='2004') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Partition Information +Partition Value: [2004] +Database: default +Table: outputtbl1 +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2004 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2004 +#### A masked pattern was here #### +1 1 2004 +1 1 2004 +2 1 2004 +2 1 2004 +3 1 2004 +3 1 2004 +7 1 2004 +7 1 2004 +8 2 2004 +8 2 2004 +PREHOOK: query: explain +insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + +PREHOOK: query: insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@outputtbl2 +POSTHOOK: query: insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@outputtbl2@ds=2008-04-08 +POSTHOOK: Lineage: outputtbl2 PARTITION(ds=2008-04-08).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl2 PARTITION(ds=2008-04-08).values EXPRESSION [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: show partitions outputTbl2 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl2 +POSTHOOK: query: show partitions outputTbl2 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl2 +ds=2008-04-08 +PREHOOK: query: desc formatted outputTbl2 partition(ds='2008-04-08') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl2 +POSTHOOK: query: desc formatted outputTbl2 partition(ds='2008-04-08') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl2 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Partition Information +Partition Value: [2008-04-08] +Database: default +Table: outputtbl2 +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 6812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: explain insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + Map 3 + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + Reducer 2 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl3 + Reducer 4 + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl3 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + hr + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl3 + +PREHOOK: query: insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@outputtbl3 +POSTHOOK: query: insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@outputtbl3@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@outputtbl3@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=11).values EXPRESSION [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=12).values EXPRESSION [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: show partitions outputTbl3 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl3 +POSTHOOK: query: show partitions outputTbl3 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl3 +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 +PREHOOK: query: desc formatted outputTbl3 partition(ds='2008-04-08', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl3 +POSTHOOK: query: desc formatted outputTbl3 partition(ds='2008-04-08', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl3 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 11] +Database: default +Table: outputtbl3 +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 6812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_3.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_3.q.out new file mode 100644 index 0000000..11c352d --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_3.q.out @@ -0,0 +1,235 @@ +PREHOOK: query: -- This is to test the union->remove->filesink optimization +-- Union of 3 subqueries is performed (all of which are map-only queries) +-- followed by select star and a file sink. +-- There is no need for any optimization, since the whole query can be processed in +-- a single map-only job +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->remove->filesink optimization +-- Union of 3 subqueries is performed (all of which are map-only queries) +-- followed by select star and a file sink. +-- There is no need for any optimization, since the whole query can be processed in +-- a single map-only job +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Union 2 <- Map 1 (NONE), Map 3 (NONE), Map 4 (NONE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 3 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Union 2 + Vertex: Union 2 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 72 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 2 +1 3 +2 1 +2 2 +2 3 +3 1 +3 2 +3 3 +7 1 +7 2 +7 3 +8 1 +8 1 +8 2 +8 2 +8 3 +8 3 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_4.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_4.q.out new file mode 100644 index 0000000..a429f3e --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_4.q.out @@ -0,0 +1,297 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 + Stage-4 + Stage-2 depends on stages: Stage-4, Stage-3, Stage-6 + Stage-0 depends on stages: Stage-2 + Stage-3 + Stage-5 + Stage-6 depends on stages: Stage-5 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-4 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Merge + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-5 + Spark +#### A masked pattern was here #### + Vertices: + Merge + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-6 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_5.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_5.q.out new file mode 100644 index 0000000..c2c7c08 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_5.q.out @@ -0,0 +1,308 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 + Stage-4 + Stage-2 depends on stages: Stage-4, Stage-3, Stage-6 + Stage-0 depends on stages: Stage-2 + Stage-3 + Stage-5 + Stage-6 depends on stages: Stage-5 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Map 2 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-4 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Spark +#### A masked pattern was here #### + Vertices: + Merge + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-5 + Spark +#### A masked pattern was here #### + Vertices: + Merge + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-6 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 68 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_6.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_6.q.out new file mode 100644 index 0000000..1bc55f4 --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_6.q.out @@ -0,0 +1,269 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: create table outputTbl2(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl2 +POSTHOOK: query: create table outputTbl2(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl2 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +PREHOOK: type: QUERY +POSTHOOK: query: explain +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2 + Stage-0 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-3 + +STAGE PLANS: + Stage: Stage-2 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 5 <- Map 4 (GROUP) + Union 3 <- Reducer 2 (NONE), Reducer 5 (NONE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + Reducer 5 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + Union 3 + Vertex: Union 3 + + Stage: Stage-3 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-1 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + +PREHOOK: query: FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +PREHOOK: Output: default@outputtbl2 +POSTHOOK: query: FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Output: default@outputtbl2 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl2.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl2.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 +PREHOOK: query: select * from outputTbl2 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl2 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl2 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_7.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_7.q.out new file mode 100644 index 0000000..b3ec85b --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_7.q.out @@ -0,0 +1,251 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 2 <- Map 1 (GROUP) + Reducer 4 <- Map 3 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reducer 2 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 178 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_8.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_8.q.out new file mode 100644 index 0000000..d55b60d --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_8.q.out @@ -0,0 +1,260 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-0 depends on stages: Stage-2 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 3 <- Map 2 (GROUP) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Map 2 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 4 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Reducer 3 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 271 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/spark/union_remove_spark_9.q.out ql/src/test/results/clientpositive/spark/union_remove_spark_9.q.out new file mode 100644 index 0000000..5e997fd --- /dev/null +++ ql/src/test/results/clientpositive/spark/union_remove_spark_9.q.out @@ -0,0 +1,299 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which contains a union and is map-only), +-- and the other one is a map-reduce query followed by select star and a file sink. +-- There is no need for the outer union. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which contains a union and is map-only), +-- and the other one is a map-reduce query followed by select star and a file sink. +-- There is no need for the outer union. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5 + Stage-4 + Stage-2 depends on stages: Stage-4, Stage-3, Stage-6 + Stage-0 depends on stages: Stage-2 + Stage-3 + Stage-5 + Stage-6 depends on stages: Stage-5 + +STAGE PLANS: + Stage: Stage-1 + Spark + Edges: + Reducer 4 <- Map 3 (GROUP) + Union 2 <- Map 1 (NONE), Map 5 (NONE) +#### A masked pattern was here #### + Vertices: + Map 1 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Map 3 + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Map 5 + Map Operator Tree: + TableScan + alias: inputtbl1 + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Reducer 4 + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + Union 2 + Vertex: Union 2 + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-4 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-2 + Dependency Collection + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-6 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 271 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_1.q.out ql/src/test/results/clientpositive/union_remove_spark_1.q.out new file mode 100644 index 0000000..6e03d4e --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_1.q.out @@ -0,0 +1,238 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_10.q.out ql/src/test/results/clientpositive/union_remove_spark_10.q.out new file mode 100644 index 0000000..74b9e68 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_10.q.out @@ -0,0 +1,315 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where one of the sub-queries requires a map-reduce +-- job), followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The outer union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where one of the sub-queries requires a map-reduce +-- job), followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The outer union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-2 depends on stages: Stage-1 + Stage-7 depends on stages: Stage-2, Stage-8 , consists of Stage-4, Stage-3, Stage-5 + Stage-4 + Stage-0 depends on stages: Stage-4, Stage-3, Stage-6 + Stage-3 + Stage-5 + Stage-6 depends on stages: Stage-5 + Stage-8 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + Union + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-7 + Conditional Operator + + Stage: Stage-4 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-6 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-8 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 271 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_11.q.out ql/src/test/results/clientpositive/union_remove_spark_11.q.out new file mode 100644 index 0000000..e0bd498 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_11.q.out @@ -0,0 +1,294 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where also contains map only sub-queries), +-- followed by select star and a file sink. +-- There is no need for the union optimization, since the whole query can be performed +-- in a single map-only job +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which is a map-only query, and the +-- other one contains a nested union where also contains map only sub-queries), +-- followed by select star and a file sink. +-- There is no need for the union optimization, since the whole query can be performed +-- in a single map-only job +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-6 depends on stages: Stage-1 , consists of Stage-3, Stage-2, Stage-4 + Stage-3 + Stage-0 depends on stages: Stage-3, Stage-2, Stage-5 + Stage-2 + Stage-4 + Stage-5 depends on stages: Stage-4 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-6 + Conditional Operator + + Stage: Stage-3 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-4 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, 1 as values from inputTbl1 +union all +select * FROM ( + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 1 + numRows -1 + rawDataSize -1 + totalSize 115 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 2 +1 3 +2 1 +2 2 +2 3 +3 1 +3 2 +3 3 +7 1 +7 2 +7 3 +8 1 +8 1 +8 2 +8 2 +8 3 +8 3 diff --git ql/src/test/results/clientpositive/union_remove_spark_15.q.out ql/src/test/results/clientpositive/union_remove_spark_15.q.out new file mode 100644 index 0000000..902400d --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_15.q.out @@ -0,0 +1,269 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This tests demonstrates that this optimization works in the presence of dynamic partitions. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This tests demonstrates that this optimization works in the presence of dynamic partitions. + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=1 +POSTHOOK: Output: default@outputtbl1@ds=2 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=1 +ds=2 +PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +1 1 1 +2 1 1 +3 1 1 +7 1 1 +8 2 1 +PREHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +1 1 2 +2 1 2 +3 1 2 +7 1 2 +8 2 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_16.q.out ql/src/test/results/clientpositive/union_remove_spark_16.q.out new file mode 100644 index 0000000..721caa0 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_16.q.out @@ -0,0 +1,297 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- This test demonstrates that this optimization works in the presence of dynamic partitions. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- This test demonstrates that this optimization works in the presence of dynamic partitions. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-6 depends on stages: Stage-1, Stage-7 , consists of Stage-3, Stage-2, Stage-4 + Stage-3 + Stage-0 depends on stages: Stage-3, Stage-2, Stage-5 + Stage-2 + Stage-4 + Stage-5 depends on stages: Stage-4 + Stage-7 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-6 + Conditional Operator + + Stage: Stage-3 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-4 + Merge Work + merge level: block + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + + Stage: Stage-5 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-7 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, '1' as ds from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values, '2' as ds from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=1 +POSTHOOK: Output: default@outputtbl1@ds=2 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=1 +ds=2 +PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +1 1 1 +2 1 1 +3 1 1 +7 1 1 +8 2 1 +PREHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +1 1 2 +2 1 2 +3 1 2 +7 1 2 +8 2 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_17.q.out ql/src/test/results/clientpositive/union_remove_spark_17.q.out new file mode 100644 index 0000000..593ed06 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_17.q.out @@ -0,0 +1,233 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need for this optimization, since the query is a map-only query. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- and the results are written to a table using dynamic partitions. +-- There is no need for this optimization, since the query is a map-only query. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int), '2' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: string) + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int), '1' (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col2 (type: string) + sort order: + + Map-reduce partition columns: _col2 (type: string) + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: bigint), _col2 (type: string) + Reduce Operator Tree: + Extract + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, 1 as values, '1' as ds from inputTbl1 + UNION ALL + SELECT key, 2 as values, '2' as ds from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=1 +POSTHOOK: Output: default@outputtbl1@ds=2 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=1).values EXPRESSION [] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2).values EXPRESSION [] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=1 +ds=2 +PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '1' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=1 +#### A masked pattern was here #### +1 1 1 +2 1 1 +3 1 1 +7 1 1 +8 1 1 +8 1 1 +PREHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '2' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2 +#### A masked pattern was here #### +1 2 2 +2 2 2 +3 2 2 +7 2 2 +8 2 2 +8 2 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_18.q.out ql/src/test/results/clientpositive/union_remove_spark_18.q.out new file mode 100644 index 0000000..eed4394 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_18.q.out @@ -0,0 +1,309 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This test demonstrates that the optimization works with dynamic partitions irrespective of the +-- file format of the output file +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- This test demonstrates that the optimization works with dynamic partitions irrespective of the +-- file format of the output file +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), ds (type: string) + outputColumnNames: key, ds + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string), ds (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col2 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: bigint), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), ds (type: string) + outputColumnNames: key, ds + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string), ds (type: string) + mode: hash + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string), _col1 (type: string) + sort order: ++ + Map-reduce partition columns: _col0 (type: string), _col1 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col2 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string), KEY._col1 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col2 (type: bigint), _col1 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 partition (ds) +SELECT * +FROM ( + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds + UNION ALL + SELECT key, count(1) as values, ds from inputTbl1 group by key, ds +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=11 +POSTHOOK: Output: default@outputtbl1@ds=12 +POSTHOOK: Output: default@outputtbl1@ds=13 +POSTHOOK: Output: default@outputtbl1@ds=17 +POSTHOOK: Output: default@outputtbl1@ds=18 +POSTHOOK: Output: default@outputtbl1@ds=28 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=11).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=11).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=12).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=12).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=13).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=13).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=17).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=17).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=18).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=18).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=28).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=28).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: show partitions outputTbl1 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: show partitions outputTbl1 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl1 +ds=11 +ds=12 +ds=13 +ds=17 +ds=18 +ds=28 +PREHOOK: query: select * from outputTbl1 where ds = '11' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=11 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '11' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=11 +#### A masked pattern was here #### +1 1 11 +1 1 11 +PREHOOK: query: select * from outputTbl1 where ds = '18' order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=18 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds = '18' order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=18 +#### A masked pattern was here #### +8 1 18 +8 1 18 +PREHOOK: query: select * from outputTbl1 where ds is not null order by key, values, ds +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=11 +PREHOOK: Input: default@outputtbl1@ds=12 +PREHOOK: Input: default@outputtbl1@ds=13 +PREHOOK: Input: default@outputtbl1@ds=17 +PREHOOK: Input: default@outputtbl1@ds=18 +PREHOOK: Input: default@outputtbl1@ds=28 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 where ds is not null order by key, values, ds +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=11 +POSTHOOK: Input: default@outputtbl1@ds=12 +POSTHOOK: Input: default@outputtbl1@ds=13 +POSTHOOK: Input: default@outputtbl1@ds=17 +POSTHOOK: Input: default@outputtbl1@ds=18 +POSTHOOK: Input: default@outputtbl1@ds=28 +#### A masked pattern was here #### +1 1 11 +1 1 11 +2 1 12 +2 1 12 +3 1 13 +3 1 13 +7 1 17 +7 1 17 +8 1 18 +8 1 18 +8 1 28 +8 1 28 diff --git ql/src/test/results/clientpositive/union_remove_spark_19.q.out ql/src/test/results/clientpositive/union_remove_spark_19.q.out new file mode 100644 index 0000000..939687b --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_19.q.out @@ -0,0 +1,576 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +2 1 +3 1 +7 1 +8 2 +1 1 +2 1 +3 1 +7 1 +8 2 +PREHOOK: query: -- filter should be fine +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +PREHOOK: type: QUERY +POSTHOOK: query: -- filter should be fine +explain +insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 30 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = 7) (type: boolean) + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '7' (type: string) + outputColumnNames: key + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 15 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: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 30 Data size: 30 Basic stats: COMPLETE Column stats: NONE + Filter Operator + predicate: (key = 7) (type: boolean) + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: '7' (type: string) + outputColumnNames: key + Statistics: Num rows: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 15 Data size: 15 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: 15 Data size: 15 Basic stats: COMPLETE Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 7 Data size: 7 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a where a.key = 7 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +7 1 +7 1 +PREHOOK: query: -- filters and sub-queries should be fine +explain +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +PREHOOK: type: QUERY +POSTHOOK: query: -- filters and sub-queries should be fine +explain +insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: (_col0 + _col0) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (_col0 >= 7.0) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: (_col0 + _col0) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Filter Operator + predicate: (_col0 >= 7.0) (type: boolean) + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +select key, values from +( +SELECT a.key + a.key as key, a.values +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +) b where b.key >= 7 +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +14.0 1 +14.0 1 +16.0 2 +16.0 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_2.q.out ql/src/test/results/clientpositive/union_remove_spark_2.q.out new file mode 100644 index 0000000..1efdce6 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_2.q.out @@ -0,0 +1,252 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2, Stage-3 + Stage-2 is a root stage + Stage-3 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 68 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_20.q.out ql/src/test/results/clientpositive/union_remove_spark_20.q.out new file mode 100644 index 0000000..bc90c08 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_20.q.out @@ -0,0 +1,240 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, the order of the columns in the select list is different. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, the order of the columns in the select list is different. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(values bigint, key string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(values bigint, key string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col1 (type: bigint), _col0 (type: string) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.values, a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +values bigint +key string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +1 2 +1 3 +1 3 +1 7 +1 7 +2 8 +2 8 diff --git ql/src/test/results/clientpositive/union_remove_spark_21.q.out ql/src/test/results/clientpositive/union_remove_spark_21.q.out new file mode 100644 index 0000000..5734786 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_21.q.out @@ -0,0 +1,238 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, all the columns are not selected. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select and a file sink +-- However, all the columns are not selected. So, union cannot +-- be removed. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23. The union is removed, the select (which changes the order of +-- columns being selected) is pushed above the union. + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string) + outputColumnNames: _col0 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT a.key +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 20 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 +1 +2 +2 +3 +3 +7 +7 +8 +8 diff --git ql/src/test/results/clientpositive/union_remove_spark_24.q.out ql/src/test/results/clientpositive/union_remove_spark_24.q.out new file mode 100644 index 0000000..38a68cf --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_24.q.out @@ -0,0 +1,236 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- One sub-query has a double and the other sub-query has a bigint. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- One sub-query has a double and the other sub-query has a bigint. +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key double, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key double, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: EXPLAIN +INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: UDFToDouble(UDFToLong(_col0)) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: UDFToDouble(_col0) (type: double), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: INSERT OVERWRITE TABLE outputTbl1 +SELECT * FROM +( + SELECT CAST(key AS DOUBLE) AS key, count(1) as values FROM inputTbl1 group by key + UNION ALL + SELECT CAST(key AS BIGINT) AS key, count(1) as values FROM inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key double +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 60 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1.0 1 +1.0 1 +2.0 1 +2.0 1 +3.0 1 +3.0 1 +7.0 1 +7.0 1 +8.0 2 +8.0 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_25.q.out ql/src/test/results/clientpositive/union_remove_spark_25.q.out new file mode 100644 index 0000000..a522a4c --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_25.q.out @@ -0,0 +1,639 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: create table outputTbl2(key string, values bigint) partitioned by (ds string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl2 +POSTHOOK: query: create table outputTbl2(key string, values bigint) partitioned by (ds string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl2 +PREHOOK: query: create table outputTbl3(key string, values bigint) partitioned by (ds string,hr string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl3 +POSTHOOK: query: create table outputTbl3(key string, values bigint) partitioned by (ds string,hr string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl3 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + partition: + ds 2004 + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1@ds=2004 +POSTHOOK: query: insert overwrite table outputTbl1 partition(ds='2004') +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1@ds=2004 +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2004).key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1 PARTITION(ds=2004).values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 partition(ds='2004') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 partition(ds='2004') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Partition Information +Partition Value: [2004] +Database: default +Table: outputtbl1 +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +PREHOOK: Input: default@outputtbl1@ds=2004 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +POSTHOOK: Input: default@outputtbl1@ds=2004 +#### A masked pattern was here #### +1 1 2004 +1 1 2004 +2 1 2004 +2 1 2004 +3 1 2004 +3 1 2004 +7 1 2004 +7 1 2004 +8 2 2004 +8 2 2004 +PREHOOK: query: explain +insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string) + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 500 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string) + outputColumnNames: _col0, _col1, _col2 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + +PREHOOK: query: insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@outputtbl2 +POSTHOOK: query: insert overwrite table outputTbl2 partition(ds) +SELECT * +FROM ( + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 + UNION ALL + SELECT key, value, ds from srcpart where ds='2008-04-08' limit 500 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@outputtbl2@ds=2008-04-08 +POSTHOOK: Lineage: outputtbl2 PARTITION(ds=2008-04-08).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl2 PARTITION(ds=2008-04-08).values EXPRESSION [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: show partitions outputTbl2 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl2 +POSTHOOK: query: show partitions outputTbl2 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl2 +ds=2008-04-08 +PREHOOK: query: desc formatted outputTbl2 partition(ds='2008-04-08') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl2 +POSTHOOK: query: desc formatted outputTbl2 partition(ds='2008-04-08') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl2 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string + +# Detailed Partition Information +Partition Value: [2008-04-08] +Database: default +Table: outputtbl2 +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 6812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: explain insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl3 + + Stage: Stage-0 + Move Operator + tables: + partition: + ds + hr + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl3 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: srcpart + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: key (type: string), value (type: string), ds (type: string), hr (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Reduce Output Operator + sort order: + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string), _col3 (type: string) + Reduce Operator Tree: + Select Operator + expressions: VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string), VALUE._col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Limit + Number of rows: 1000 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint), _col2 (type: string), _col3 (type: string) + outputColumnNames: _col0, _col1, _col2, _col3 + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 58 Data size: 11624 Basic stats: COMPLETE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl3 + +PREHOOK: query: insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@srcpart +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +PREHOOK: Output: default@outputtbl3 +POSTHOOK: query: insert overwrite table outputTbl3 partition(ds, hr) +SELECT * +FROM ( + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 + UNION ALL + SELECT key, value, ds, hr from srcpart where ds='2008-04-08' limit 1000 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@srcpart +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=11 +POSTHOOK: Input: default@srcpart@ds=2008-04-08/hr=12 +POSTHOOK: Output: default@outputtbl3@ds=2008-04-08/hr=11 +POSTHOOK: Output: default@outputtbl3@ds=2008-04-08/hr=12 +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=11).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=11).values EXPRESSION [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=12).key EXPRESSION [(srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:key, type:string, comment:default), ] +POSTHOOK: Lineage: outputtbl3 PARTITION(ds=2008-04-08,hr=12).values EXPRESSION [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), (srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ] +PREHOOK: query: show partitions outputTbl3 +PREHOOK: type: SHOWPARTITIONS +PREHOOK: Input: default@outputtbl3 +POSTHOOK: query: show partitions outputTbl3 +POSTHOOK: type: SHOWPARTITIONS +POSTHOOK: Input: default@outputtbl3 +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 +PREHOOK: query: desc formatted outputTbl3 partition(ds='2008-04-08', hr='11') +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl3 +POSTHOOK: query: desc formatted outputTbl3 partition(ds='2008-04-08', hr='11') +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl3 +# col_name data_type comment + +key string +values bigint + +# Partition Information +# col_name data_type comment + +ds string +hr string + +# Detailed Partition Information +Partition Value: [2008-04-08, 11] +Database: default +Table: outputtbl3 +#### A masked pattern was here #### +Protect Mode: None +#### A masked pattern was here #### +Partition Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 6812 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git ql/src/test/results/clientpositive/union_remove_spark_3.q.out ql/src/test/results/clientpositive/union_remove_spark_3.q.out new file mode 100644 index 0000000..bda484b --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_3.q.out @@ -0,0 +1,238 @@ +PREHOOK: query: -- This is to test the union->remove->filesink optimization +-- Union of 3 subqueries is performed (all of which are map-only queries) +-- followed by select star and a file sink. +-- There is no need for any optimization, since the whole query can be processed in +-- a single map-only job +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->remove->filesink optimization +-- Union of 3 subqueries is performed (all of which are map-only queries) +-- followed by select star and a file sink. +-- There is no need for any optimization, since the whole query can be processed in +-- a single map-only job +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1 + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 3 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 90 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 + UNION ALL + SELECT key, 3 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 1 + numRows -1 + rawDataSize -1 + totalSize 72 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 2 +1 3 +2 1 +2 2 +2 3 +3 1 +3 2 +3 3 +7 1 +7 2 +7 3 +8 1 +8 1 +8 2 +8 2 +8 3 +8 3 diff --git ql/src/test/results/clientpositive/union_remove_spark_4.q.out ql/src/test/results/clientpositive/union_remove_spark_4.q.out new file mode 100644 index 0000000..f27f6b7 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_4.q.out @@ -0,0 +1,282 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-6 depends on stages: Stage-1, Stage-7 , consists of Stage-3, Stage-2, Stage-4 + Stage-3 + Stage-0 depends on stages: Stage-3, Stage-2, Stage-5 + Stage-2 + Stage-4 + Stage-5 depends on stages: Stage-4 + Stage-7 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-6 + Conditional Operator + + Stage: Stage-3 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-4 + Map Reduce + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-5 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-7 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 40 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_5.q.out ql/src/test/results/clientpositive/union_remove_spark_5.q.out new file mode 100644 index 0000000..a6d7b3d --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_5.q.out @@ -0,0 +1,298 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-6 depends on stages: Stage-1, Stage-7, Stage-8 , consists of Stage-3, Stage-2, Stage-4 + Stage-3 + Stage-0 depends on stages: Stage-3, Stage-2, Stage-5 + Stage-2 + Stage-4 + Stage-5 depends on stages: Stage-4 + Stage-7 is a root stage + Stage-8 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-6 + Conditional Operator + + Stage: Stage-3 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-4 + Map Reduce + Map Operator Tree: + TableScan + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-5 + Move Operator + files: + hdfs directory: true +#### A masked pattern was here #### + + Stage: Stage-7 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-8 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 68 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_6.q.out ql/src/test/results/clientpositive/union_remove_spark_6.q.out new file mode 100644 index 0000000..a9d2e8b --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_6.q.out @@ -0,0 +1,292 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (all of which are mapred queries) +-- followed by select star and a file sink in 2 output tables. +-- The optimiaztion does not take affect since it is a multi-table insert. +-- It does not matter, whether the output is merged or not. In this case, +-- merging is turned off + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: create table outputTbl2(key string, values bigint) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl2 +POSTHOOK: query: create table outputTbl2(key string, values bigint) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl2 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +PREHOOK: type: QUERY +POSTHOOK: query: explain +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-2 is a root stage + Stage-3 depends on stages: Stage-2, Stage-4 + Stage-0 depends on stages: Stage-3 + Stage-1 depends on stages: Stage-3 + Stage-4 is a root stage + +STAGE PLANS: + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + + Stage: Stage-3 + Map Reduce + Map Operator Tree: + TableScan + Union + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + TableScan + Union + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl1 + + Stage: Stage-1 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.mapred.TextInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe + name: default.outputtbl2 + + Stage: Stage-4 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe + +PREHOOK: query: FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +PREHOOK: Output: default@outputtbl2 +POSTHOOK: query: FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +insert overwrite table outputTbl1 select * +insert overwrite table outputTbl2 select * +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Output: default@outputtbl2 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +POSTHOOK: Lineage: outputtbl2.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl2.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 +PREHOOK: query: select * from outputTbl2 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl2 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl2 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl2 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_7.q.out ql/src/test/results/clientpositive/union_remove_spark_7.q.out new file mode 100644 index 0000000..6a28b95 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_7.q.out @@ -0,0 +1,242 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 map-reduce subqueries is performed followed by select star and a file sink +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, count(1) as values from inputTbl1 group by key +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 178 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +2 1 +2 1 +3 1 +3 1 +7 1 +7 1 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_8.q.out ql/src/test/results/clientpositive/union_remove_spark_8.q.out new file mode 100644 index 0000000..a688cff --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_8.q.out @@ -0,0 +1,256 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 3 subqueries is performed (exactly one of which requires a map-reduce job) +-- followed by select star and a file sink. +-- There is no need to write the temporary results of the sub-queries, and then read them +-- again to process the union. The union can be removed completely. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- off + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2, Stage-3 + Stage-2 is a root stage + Stage-3 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(2) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-3 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), UDFToLong(1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * +FROM ( + SELECT key, count(1) as values from inputTbl1 group by key + UNION ALL + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 3 + numRows -1 + rawDataSize -1 + totalSize 271 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 diff --git ql/src/test/results/clientpositive/union_remove_spark_9.q.out ql/src/test/results/clientpositive/union_remove_spark_9.q.out new file mode 100644 index 0000000..69b2e64 --- /dev/null +++ ql/src/test/results/clientpositive/union_remove_spark_9.q.out @@ -0,0 +1,269 @@ +PREHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which contains a union and is map-only), +-- and the other one is a map-reduce query followed by select star and a file sink. +-- There is no need for the outer union. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@inputTbl1 +POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization +-- Union of 2 subqueries is performed (one of which contains a union and is map-only), +-- and the other one is a map-reduce query followed by select star and a file sink. +-- There is no need for the outer union. +-- The final file format is different from the input and intermediate file format. +-- It does not matter, whether the output is merged or not. In this case, merging is turned +-- on + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) +-- Since this test creates sub-directories for the output table outputTbl1, it might be easier +-- to run the test only on hadoop 23 + +create table inputTbl1(key string, val string) stored as textfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@inputTbl1 +PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@outputTbl1 +POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@outputTbl1 +PREHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +PREHOOK: type: LOAD +#### A masked pattern was here #### +PREHOOK: Output: default@inputtbl1 +POSTHOOK: query: load data local inpath '../../data/files/T1.txt' into table inputTbl1 +POSTHOOK: type: LOAD +#### A masked pattern was here #### +POSTHOOK: Output: default@inputtbl1 +PREHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +POSTHOOK: query: explain +insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +STAGE DEPENDENCIES: + Stage-1 is a root stage + Stage-0 depends on stages: Stage-1, Stage-2 + Stage-2 is a root stage + +STAGE PLANS: + Stage: Stage-1 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 1 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string), 2 (type: int) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Union + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: _col0 (type: string), UDFToLong(_col1) (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 60 Basic stats: PARTIAL Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-0 + Move Operator + tables: + replace: true + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + + Stage: Stage-2 + Map Reduce + Map Operator Tree: + TableScan + alias: inputtbl1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Select Operator + expressions: key (type: string) + outputColumnNames: key + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Group By Operator + aggregations: count(1) + keys: key (type: string) + mode: hash + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + Reduce Output Operator + key expressions: _col0 (type: string) + sort order: + + Map-reduce partition columns: _col0 (type: string) + Statistics: Num rows: 0 Data size: 30 Basic stats: PARTIAL Column stats: NONE + value expressions: _col1 (type: bigint) + Reduce Operator Tree: + Group By Operator + aggregations: count(VALUE._col0) + keys: KEY._col0 (type: string) + mode: mergepartial + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + Select Operator + expressions: _col0 (type: string), _col1 (type: bigint) + outputColumnNames: _col0, _col1 + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + File Output Operator + compressed: false + Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE + table: + input format: org.apache.hadoop.hive.ql.io.RCFileInputFormat + output format: org.apache.hadoop.hive.ql.io.RCFileOutputFormat + serde: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe + name: default.outputtbl1 + +PREHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +PREHOOK: type: QUERY +PREHOOK: Input: default@inputtbl1 +PREHOOK: Output: default@outputtbl1 +POSTHOOK: query: insert overwrite table outputTbl1 +SELECT * FROM +( +select key, count(1) as values from inputTbl1 group by key +union all +select * FROM ( + SELECT key, 1 as values from inputTbl1 + UNION ALL + SELECT key, 2 as values from inputTbl1 +) a +)b +POSTHOOK: type: QUERY +POSTHOOK: Input: default@inputtbl1 +POSTHOOK: Output: default@outputtbl1 +POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ] +POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, ] +PREHOOK: query: desc formatted outputTbl1 +PREHOOK: type: DESCTABLE +PREHOOK: Input: default@outputtbl1 +POSTHOOK: query: desc formatted outputTbl1 +POSTHOOK: type: DESCTABLE +POSTHOOK: Input: default@outputtbl1 +# col_name data_type comment + +key string +values bigint + +# Detailed Table Information +Database: default +#### A masked pattern was here #### +Protect Mode: None +Retention: 0 +#### A masked pattern was here #### +Table Type: MANAGED_TABLE +Table Parameters: + COLUMN_STATS_ACCURATE false + numFiles 2 + numRows -1 + rawDataSize -1 + totalSize 192 +#### A masked pattern was here #### + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe +InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 +PREHOOK: query: select * from outputTbl1 order by key, values +PREHOOK: type: QUERY +PREHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +POSTHOOK: query: select * from outputTbl1 order by key, values +POSTHOOK: type: QUERY +POSTHOOK: Input: default@outputtbl1 +#### A masked pattern was here #### +1 1 +1 1 +1 2 +2 1 +2 1 +2 2 +3 1 +3 1 +3 2 +7 1 +7 1 +7 2 +8 1 +8 1 +8 2 +8 2 +8 2 -- 1.8.5.2 (Apple Git-48)