Index: conf/hive-default.xml.template
===================================================================
--- conf/hive-default.xml.template (revision 1403193)
+++ conf/hive-default.xml.template (working copy)
@@ -453,6 +453,29 @@
+ hive.optimize.union.remove
+ false
+
+ Whether to remove the union and push the operators between union and the filesink above
+ union. This avoids an extra scan of the output by union. This is independently useful for union
+ queries, and specially useful when hive.optimize.skewjoin.compiletime is set to true, since an
+ extra union is inserted.
+
+ The merge is triggered if either of hive.merge.mapfiles or hive.merge.mapredfiles is set to true.
+ If the user has set hive.merge.mapfiles to true and hive.merge.mapredfiles to false, the idea was the
+ number of reducers are few, so the number of files anyway are small. However, with this optimization,
+ we are increasing the number of files possibly by a big margin. So, we merge aggresively.
+
+
+
+ hive.mapred.supports.subdirectories
+ false
+ Whether the version of hadoop which is running supports sub-directories for tables/partitions.
+ Many hive optimizations can be applied if the hadoop version supports sub-directories for
+ tables/partitions. It was added by MAPREDUCE-1501
+
+
+
hive.multigroupby.singlemr
false
Whether to optimize multi group by query to generate single M/R
Index: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
===================================================================
--- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (revision 1403193)
+++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (working copy)
@@ -510,7 +510,16 @@
HIVEOPTBUCKETMAPJOIN("hive.optimize.bucketmapjoin", false), // optimize bucket map join
HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false), // try to use sorted merge bucket map join
HIVEOPTREDUCEDEDUPLICATION("hive.optimize.reducededuplication", true),
+ // whether to optimize union followed by select followed by filesink
+ // It creates sub-directories in the final output, so should not be turned on in systems
+ // where MAPREDUCE-1501 is not present
+ HIVE_OPTIMIZE_UNION_REMOVE("hive.optimize.union.remove", false),
+ // whether hadoop map-reduce supports sub-directories. It was added by MAPREDUCE-1501.
+ // Some optimizations can only be performed if the version of hadoop being used supports
+ // sub-directories
+ HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES("hive.mapred.supports.subdirectories", false),
+
// optimize skewed join by changing the query plan at compile time
HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME("hive.optimize.skewjoin.compiletime", false),
Index: ql/src/test/results/clientpositive/union_remove_17.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_17.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_17.q.out (working copy)
@@ -0,0 +1,266 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values) (TOK_SELEXPR '1' ds)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values) (TOK_SELEXPR '2' ds))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1) (TOK_PARTSPEC (TOK_PARTVAL ds)))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ expr: '1'
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ expr: '2'
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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 []
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+
+# 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
+POSTHOOK: query: show partitions outputTbl1
+POSTHOOK: type: SHOWPARTITIONS
+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 []
+ds=1
+ds=2
+PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values
+PREHOOK: type: QUERY
+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@ds=1
+#### A masked pattern was here ####
+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 []
+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@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@ds=2
+#### A masked pattern was here ####
+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 []
+1 2 2
+2 2 2
+3 2 2
+7 2 2
+8 2 2
+8 2 2
Index: ql/src/test/results/clientpositive/union_remove_12.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_12.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_12.q.out (working copy)
@@ -0,0 +1,320 @@
+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 is a map-join query), followed by select star and a file sink.
+-- The union optimization is applied, and the union is removed.
+
+-- 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 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
+
+create table inputTbl1(key string, val string) stored as textfile
+PREHOOK: type: CREATETABLE
+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 is a map-join query), followed by select star and a file sink.
+-- The union optimization is applied, and the union is removed.
+
+-- 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 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
+
+create table inputTbl1(key string, val string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@inputtbl1
+PREHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT * FROM
+(
+select key, 1 as values from inputTbl1
+union all
+select /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT * FROM
+(
+select key, 1 as values from inputTbl1
+union all
+select /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values)))) (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME inputTbl1) a) (TOK_TABREF (TOK_TABNAME inputTbl1) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) val) values))))) c)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-9 is a root stage
+ Stage-7 depends on stages: Stage-2, Stage-9 , 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-10 is a root stage
+ Stage-1 depends on stages: Stage-10
+ Stage-2 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-9
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1:c-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToString(_col1)
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Block level merge
+
+ Stage: Stage-5
+ Block level merge
+
+ Stage: Stage-6
+ Move Operator
+ files:
+ hdfs directory: true
+#### A masked pattern was here ####
+
+ Stage: Stage-10
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ null-subquery2:c-subquery2:a
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ null-subquery2:c-subquery2:a
+ TableScan
+ alias: a
+ HashTable Sink Operator
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ Position of Big Table: 1
+
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery2:c-subquery2:b
+ TableScan
+ alias: b
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ outputColumnNames: _col0, _col5
+ Position of Big Table: 1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ Local Work:
+ Map Reduce Local Work
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@inputtbl1
+POSTHOOK: Output: default@outputtbl1
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ]
+PREHOOK: query: desc formatted outputTbl1
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ]
+1 1
+1 11
+2 1
+2 12
+3 1
+3 13
+7 1
+7 17
+8 1
+8 1
+8 18
+8 18
+8 28
+8 28
Index: ql/src/test/results/clientpositive/union_remove_5.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_5.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_5.q.out (working copy)
@@ -0,0 +1,350 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values))))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ 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
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1-subquery1:a-subquery1-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1-subquery2:a-subquery1-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+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
Index: ql/src/test/results/clientpositive/union_remove_23.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_23.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_23.q.out (working copy)
@@ -0,0 +1,333 @@
+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 of the sub-queries
+-- would have multiple map-reduce jobs.
+-- 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
+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 of the sub-queries
+-- would have multiple map-reduce jobs.
+-- 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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@inputtbl1
+PREHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT *
+FROM (
+ SELECT key, count(1) as values from
+ (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key
+ UNION ALL
+ SELECT key, count(1) as values from inputTbl1 group by key
+) subq2
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT *
+FROM (
+ SELECT key, count(1) as values from
+ (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key
+ UNION ALL
+ SELECT key, count(1) as values from inputTbl1 group by key
+) subq2
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME inputTbl1) a) (TOK_TABREF (TOK_TABNAME inputTbl1) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) val))))) subq)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) subq2)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1, Stage-3
+ Stage-2 is a root stage
+ Stage-3 depends on stages: Stage-2
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery2:subq2-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:subq2-subquery1:subq:a
+ TableScan
+ alias: a
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ null-subquery1:subq2-subquery1:subq:b
+ TableScan
+ alias: b
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1
+ handleSkewJoin: false
+ outputColumnNames: _col0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: _col0
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-3
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key
+ UNION ALL
+ SELECT key, count(1) as values from inputTbl1 group by key
+) subq2
+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
+ (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key
+ UNION ALL
+ SELECT key, count(1) as values from inputTbl1 group by key
+) subq2
+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)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)b.null, (inputtbl1)a.null, (inputtbl1)b.null, ]
+PREHOOK: query: desc formatted outputTbl1
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)b.null, (inputtbl1)a.null, (inputtbl1)b.null, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)b.null, (inputtbl1)a.null, (inputtbl1)b.null, ]
+1 1
+1 1
+2 1
+2 1
+3 1
+3 1
+7 1
+7 1
+8 2
+8 4
Index: ql/src/test/results/clientpositive/skewjoin_union_remove_2.q.out
===================================================================
--- ql/src/test/results/clientpositive/skewjoin_union_remove_2.q.out (revision 0)
+++ ql/src/test/results/clientpositive/skewjoin_union_remove_2.q.out (working copy)
@@ -0,0 +1,290 @@
+PREHOOK: query: CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@T1
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@t1
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@t1
+PREHOOK: query: CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@T2
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2
+PREHOOK: type: LOAD
+PREHOOK: Output: default@t2
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@t2
+PREHOOK: query: CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@T3
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3
+PREHOOK: type: LOAD
+PREHOOK: Output: default@t3
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@t3
+PREHOOK: query: -- This is to test the union->selectstar->filesink and skewjoin optimization
+-- Union of 3 map-reduce subqueries is performed for the skew join
+-- 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.
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+-- Since this test creates sub-directories for the output table, it might be easier
+-- to run the test only on hadoop 23
+
+EXPLAIN
+SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key
+PREHOOK: type: QUERY
+POSTHOOK: query: -- This is to test the union->selectstar->filesink and skewjoin optimization
+-- Union of 3 map-reduce subqueries is performed for the skew join
+-- 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.
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+-- Since this test creates sub-directories for the output table, it might be easier
+-- to run the test only on hadoop 23
+
+EXPLAIN
+SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key))) (TOK_TABREF (TOK_TABNAME T3) c) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL c) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME b))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME c))))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-4 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ subquery1:a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: (not (((key = '2') or (key = '8')) or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ subquery1:b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: (not (((key = '2') or (key = '8')) or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ subquery1:c
+ TableScan
+ alias: c
+ Filter Operator
+ predicate:
+ expr: (not (((key = '2') or (key = '8')) or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 2
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ Inner Join 0 to 2
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ 2 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5, _col8, _col9
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ expr: _col8
+ type: string
+ expr: _col9
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-4
+ Map Reduce
+ Alias -> Map Operator Tree:
+ a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: (((key = '2') or (key = '8')) or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: (((key = '2') or (key = '8')) or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ c
+ TableScan
+ alias: c
+ Filter Operator
+ predicate:
+ expr: (((key = '2') or (key = '8')) or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 2
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ Inner Join 0 to 2
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ 2 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5, _col8, _col9
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ expr: _col8
+ type: string
+ expr: _col9
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key
+ORDER BY a.key, b.key, c.key, a.val, b.val, c.val
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+PREHOOK: Input: default@t3
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key
+ORDER BY a.key, b.key, c.key, a.val, b.val, c.val
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+POSTHOOK: Input: default@t3
+#### A masked pattern was here ####
+2 12 2 22 2 12
Index: ql/src/test/results/clientpositive/union_remove_16.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_16.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_16.q.out (working copy)
@@ -0,0 +1,351 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values) (TOK_SELEXPR '1' ds)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values) (TOK_SELEXPR '2' ds)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1) (TOK_PARTSPEC (TOK_PARTVAL ds)))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: '2'
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Block level merge
+
+ Stage: Stage-4
+ Block level merge
+
+ Stage: Stage-5
+ Move Operator
+ files:
+ hdfs directory: true
+#### A masked pattern was here ####
+
+ Stage: Stage-7
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: '1'
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+
+# 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
+POSTHOOK: query: show partitions outputTbl1
+POSTHOOK: type: SHOWPARTITIONS
+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, ]
+ds=1
+ds=2
+PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values
+PREHOOK: type: QUERY
+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@ds=1
+#### A masked pattern was here ####
+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, ]
+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@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@ds=2
+#### A masked pattern was here ####
+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, ]
+1 1 2
+2 1 2
+3 1 2
+7 1 2
+8 2 2
Index: ql/src/test/results/clientpositive/union_remove_11.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_11.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_11.q.out (working copy)
@@ -0,0 +1,323 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values)))) (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 3 values))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))) b)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery1:b-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ null-subquery2:b-subquery2-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ null-subquery2:b-subquery2-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 3
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Block level merge
+
+ Stage: Stage-4
+ Block level merge
+
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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 []
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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 []
+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
Index: ql/src/test/results/clientpositive/union_remove_9.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_9.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_9.q.out (working copy)
@@ -0,0 +1,335 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))) b)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:b-subquery2-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ null-subquery2:b-subquery2-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Block level merge
+
+ Stage: Stage-4
+ Block level merge
+
+ Stage: Stage-5
+ Move Operator
+ files:
+ hdfs directory: true
+#### A masked pattern was here ####
+
+ Stage: Stage-7
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1:b-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+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
Index: ql/src/test/results/clientpositive/union_remove_4.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_4.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_4.q.out (working copy)
@@ -0,0 +1,323 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ 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
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+1 1
+1 1
+2 1
+2 1
+3 1
+3 1
+7 1
+7 1
+8 2
+8 2
Index: ql/src/test/results/clientpositive/union_remove_22.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_22.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_22.q.out (working copy)
@@ -0,0 +1,528 @@
+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, some columns are repeated. 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 selects columns from
+-- both the sub-qeuries of the union) is pushed above the union.
+
+create table inputTbl1(key string, val string) stored as textfile
+PREHOOK: type: CREATETABLE
+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, some columns are repeated. 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 selects columns from
+-- both the sub-qeuries of the union) is pushed above the union.
+
+create table inputTbl1(key string, val string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint, values2 bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint, values2 bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@inputtbl1
+PREHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT a.key, a.values, 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, 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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) values)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) values)))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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, 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, 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, ]
+POSTHOOK: Lineage: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+PREHOOK: query: desc formatted outputTbl1
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+values2 bigint None
+
+# 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: 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 ####
+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: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+1 1 1
+2 1 1
+3 1 1
+7 1 1
+8 2 2
+1 1 1
+2 1 1
+3 1 1
+7 1 1
+8 2 2
+PREHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT a.key, concat(a.values, a.values), concat(a.values, 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, concat(a.values, a.values), concat(a.values, 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: 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: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL a) values) (. (TOK_TABLE_OR_COL a) values))) (TOK_SELEXPR (TOK_FUNCTION concat (. (TOK_TABLE_OR_COL a) values) (. (TOK_TABLE_OR_COL a) values))))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: concat(_col1, _col1)
+ type: string
+ expr: concat(_col1, _col1)
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ expr: UDFToLong(_col2)
+ type: bigint
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: concat(_col1, _col1)
+ type: string
+ expr: concat(_col1, _col1)
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ expr: UDFToLong(_col2)
+ type: bigint
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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, concat(a.values, a.values), concat(a.values, 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, concat(a.values, a.values), concat(a.values, 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.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: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+POSTHOOK: Lineage: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+POSTHOOK: Lineage: outputtbl1.values2 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 ####
+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.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: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+POSTHOOK: Lineage: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+POSTHOOK: Lineage: outputtbl1.values2 EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+1 11 11
+1 11 11
+2 11 11
+2 11 11
+3 11 11
+3 11 11
+7 11 11
+7 11 11
+8 22 22
+8 22 22
Index: ql/src/test/results/clientpositive/skewjoin_union_remove_1.q.out
===================================================================
--- ql/src/test/results/clientpositive/skewjoin_union_remove_1.q.out (revision 0)
+++ ql/src/test/results/clientpositive/skewjoin_union_remove_1.q.out (working copy)
@@ -0,0 +1,875 @@
+PREHOOK: query: -- This is to test the union->selectstar->filesink and skewjoin optimization
+-- Union of 2 map-reduce subqueries is performed for the skew join
+-- 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.
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+-- Since this test creates sub-directories for the output, it might be easier to run the test
+-- only on hadoop 23
+
+CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2)) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- This is to test the union->selectstar->filesink and skewjoin optimization
+-- Union of 2 map-reduce subqueries is performed for the skew join
+-- 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.
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+-- Since this test creates sub-directories for the output, it might be easier to run the test
+-- only on hadoop 23
+
+CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2)) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@T1
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@t1
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@t1
+PREHOOK: query: CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3)) STORED AS TEXTFILE
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3)) STORED AS TEXTFILE
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@T2
+PREHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2
+PREHOOK: type: LOAD
+PREHOOK: Output: default@t2
+POSTHOOK: query: LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@t2
+PREHOOK: query: -- a simple join query with skew on both the tables on the join key
+
+EXPLAIN
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: -- a simple join query with skew on both the tables on the join key
+
+EXPLAIN
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-3 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ subquery1:a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ subquery1:b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-3
+ Map Reduce
+ Alias -> Map Operator Tree:
+ a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+ORDER BY a.key, b.key, a.val, b.val
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+ORDER BY a.key, b.key, a.val, b.val
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+#### A masked pattern was here ####
+2 12 2 22
+3 13 3 13
+8 18 8 18
+8 18 8 18
+8 28 8 18
+8 28 8 18
+PREHOOK: query: -- test outer joins also
+
+EXPLAIN
+SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: -- test outer joins also
+
+EXPLAIN
+SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME a))) (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME b))))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-3 is a root stage
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ subquery1:a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ subquery1:b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Right Outer Join0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-3
+ Map Reduce
+ Alias -> Map Operator Tree:
+ a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Right Outer Join0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+
+
+PREHOOK: query: SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+ORDER BY a.key, b.key, a.val, b.val
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+ORDER BY a.key, b.key, a.val, b.val
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+#### A masked pattern was here ####
+NULL NULL 4 14
+NULL NULL 5 15
+2 12 2 22
+3 13 3 13
+8 18 8 18
+8 18 8 18
+8 28 8 18
+8 28 8 18
+PREHOOK: query: create table DEST1(key1 STRING, val1 STRING, key2 STRING, val2 STRING)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table DEST1(key1 STRING, val1 STRING, key2 STRING, val2 STRING)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@DEST1
+PREHOOK: query: EXPLAIN
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME DEST1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1, Stage-3
+ Stage-3 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ subquery1:a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ subquery1:b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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.dest1
+
+ 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.dest1
+
+ Stage: Stage-3
+ Map Reduce
+ Alias -> Map Operator Tree:
+ a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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.dest1
+
+
+PREHOOK: query: INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+PREHOOK: Output: default@dest1
+POSTHOOK: query: INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+POSTHOOK: Output: default@dest1
+POSTHOOK: Lineage: dest1.key1 SIMPLE [(t1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key2 SIMPLE [(t2)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val1 SIMPLE [(t1)a.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val2 SIMPLE [(t2)b.FieldSchema(name:val, type:string, comment:null), ]
+PREHOOK: query: SELECT * FROM DEST1
+ORDER BY key1, key2, val1, val2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dest1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM DEST1
+ORDER BY key1, key2, val1, val2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dest1
+#### A masked pattern was here ####
+POSTHOOK: Lineage: dest1.key1 SIMPLE [(t1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key2 SIMPLE [(t2)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val1 SIMPLE [(t1)a.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val2 SIMPLE [(t2)b.FieldSchema(name:val, type:string, comment:null), ]
+2 12 2 22
+3 13 3 13
+8 18 8 18
+8 18 8 18
+8 28 8 18
+8 28 8 18
+PREHOOK: query: EXPLAIN
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Lineage: dest1.key1 SIMPLE [(t1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key2 SIMPLE [(t2)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val1 SIMPLE [(t1)a.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val2 SIMPLE [(t2)b.FieldSchema(name:val, type:string, comment:null), ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_RIGHTOUTERJOIN (TOK_TABREF (TOK_TABNAME T1) a) (TOK_TABREF (TOK_TABNAME T2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME DEST1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1, Stage-3
+ Stage-3 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ subquery1:a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ subquery1:b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: (not ((key = '2') or (key = '3')))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Right Outer Join0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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.dest1
+
+ 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.dest1
+
+ Stage: Stage-3
+ Map Reduce
+ Alias -> Map Operator Tree:
+ a
+ TableScan
+ alias: a
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ b
+ TableScan
+ alias: b
+ Filter Operator
+ predicate:
+ expr: ((key = '2') or (key = '3'))
+ type: boolean
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: key
+ type: string
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Right Outer Join0 to 1
+ condition expressions:
+ 0 {VALUE._col0} {VALUE._col1}
+ 1 {VALUE._col0} {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col1, _col4, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ expr: _col4
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Select Operator
+ SELECT * : (no compute)
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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.dest1
+
+
+PREHOOK: query: INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+PREHOOK: type: QUERY
+PREHOOK: Input: default@t1
+PREHOOK: Input: default@t2
+PREHOOK: Output: default@dest1
+POSTHOOK: query: INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@t1
+POSTHOOK: Input: default@t2
+POSTHOOK: Output: default@dest1
+POSTHOOK: Lineage: dest1.key1 SIMPLE [(t1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key1 SIMPLE [(t1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key2 SIMPLE [(t2)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key2 SIMPLE [(t2)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val1 SIMPLE [(t1)a.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val1 SIMPLE [(t1)a.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val2 SIMPLE [(t2)b.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val2 SIMPLE [(t2)b.FieldSchema(name:val, type:string, comment:null), ]
+PREHOOK: query: SELECT * FROM DEST1
+ORDER BY key1, key2, val1, val2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@dest1
+#### A masked pattern was here ####
+POSTHOOK: query: SELECT * FROM DEST1
+ORDER BY key1, key2, val1, val2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@dest1
+#### A masked pattern was here ####
+POSTHOOK: Lineage: dest1.key1 SIMPLE [(t1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key1 SIMPLE [(t1)a.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key2 SIMPLE [(t2)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.key2 SIMPLE [(t2)b.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val1 SIMPLE [(t1)a.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val1 SIMPLE [(t1)a.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val2 SIMPLE [(t2)b.FieldSchema(name:val, type:string, comment:null), ]
+POSTHOOK: Lineage: dest1.val2 SIMPLE [(t2)b.FieldSchema(name:val, type:string, comment:null), ]
+NULL NULL 4 14
+NULL NULL 5 15
+2 12 2 22
+3 13 3 13
+8 18 8 18
+8 18 8 18
+8 28 8 18
+8 28 8 18
Index: ql/src/test/results/clientpositive/union_remove_15.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_15.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_15.q.out (working copy)
@@ -0,0 +1,327 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values) (TOK_SELEXPR '1' ds)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values) (TOK_SELEXPR '2' ds)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1) (TOK_PARTSPEC (TOK_PARTVAL ds)))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: '2'
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: '1'
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+
+# 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
+POSTHOOK: query: show partitions outputTbl1
+POSTHOOK: type: SHOWPARTITIONS
+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, ]
+ds=1
+ds=2
+PREHOOK: query: select * from outputTbl1 where ds = '1' order by key, values
+PREHOOK: type: QUERY
+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@ds=1
+#### A masked pattern was here ####
+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, ]
+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@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@ds=2
+#### A masked pattern was here ####
+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, ]
+1 1 2
+2 1 2
+3 1 2
+7 1 2
+8 2 2
Index: ql/src/test/results/clientpositive/union_remove_10.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_10.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_10.q.out (working copy)
@@ -0,0 +1,366 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values)))) (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))) b)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:b-subquery2-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ TableScan
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ null-subquery2:b-subquery2-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Block level merge
+
+ Stage: Stage-5
+ Block level merge
+
+ Stage: Stage-6
+ Move Operator
+ files:
+ hdfs directory: true
+#### A masked pattern was here ####
+
+ Stage: Stage-8
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1:b-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+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
Index: ql/src/test/results/clientpositive/union_remove_8.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_8.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_8.q.out (working copy)
@@ -0,0 +1,306 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values))))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1-subquery1:a-subquery1-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1-subquery2:a-subquery1-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+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
Index: ql/src/test/results/clientpositive/union_remove_3.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_3.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_3.q.out (working copy)
@@ -0,0 +1,267 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values))))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 3 values))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1-subquery1:a-subquery1-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ null-subquery1-subquery2:a-subquery1-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 3
+ type: int
+ outputColumnNames: _col0, _col1
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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 []
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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 []
+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
Index: ql/src/test/results/clientpositive/union_remove_21.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_21.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_21.q.out (working copy)
@@ -0,0 +1,271 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+# col_name data_type comment
+
+key string None
+
+# 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: 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 ####
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+1
+1
+2
+2
+3
+3
+7
+7
+8
+8
Index: ql/src/test/results/clientpositive/union_remove_19.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_19.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_19.q.out (working copy)
@@ -0,0 +1,747 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) values)))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+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
+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, ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) values))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL a) key) 7))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Filter Operator
+ predicate:
+ expr: (key = 7.0)
+ type: boolean
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Filter Operator
+ predicate:
+ expr: (key = 7.0)
+ type: boolean
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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.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: 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 ####
+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.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: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+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
+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.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: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (+ (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL a) key)) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) values))))) b)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_TABLE_OR_COL values))) (TOK_WHERE (>= (. (TOK_TABLE_OR_COL b) key) 7))))
+
+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
+ Alias -> Map Operator Tree:
+ b-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: (_col0 + _col0)
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Filter Operator
+ predicate:
+ expr: (_col0 >= 7.0)
+ type: boolean
+ Select Operator
+ expressions:
+ expr: _col0
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ b-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: (_col0 + _col0)
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Filter Operator
+ predicate:
+ expr: (_col0 >= 7.0)
+ type: boolean
+ Select Operator
+ expressions:
+ expr: _col0
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+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: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.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 ####
+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.key EXPRESSION [(inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+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: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)inputtbl1.null, (inputtbl1)inputtbl1.null, ]
+14.0 1
+14.0 1
+16.0 2
+16.0 2
Index: ql/src/test/results/clientpositive/union_remove_14.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_14.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_14.q.out (working copy)
@@ -0,0 +1,452 @@
+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 join, which should be performed as a map-join query at runtime),
+-- followed by select star and a file sink.
+-- The union selectstar optimization should be performed, and the union should be removed.
+
+-- 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 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
+
+create table inputTbl1(key string, val string) stored as textfile
+PREHOOK: type: CREATETABLE
+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 join, which should be performed as a map-join query at runtime),
+-- followed by select star and a file sink.
+-- The union selectstar optimization should be performed, and the union should be removed.
+
+-- 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 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
+
+create table inputTbl1(key string, val string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+POSTHOOK: Output: default@inputtbl1
+PREHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT * FROM
+(
+select key, 1 as values from inputTbl1
+union all
+select a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+PREHOOK: type: QUERY
+POSTHOOK: query: explain
+insert overwrite table outputTbl1
+SELECT * FROM
+(
+select key, 1 as values from inputTbl1
+union all
+select a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values)))) (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME inputTbl1) a) (TOK_TABREF (TOK_TABNAME inputTbl1) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) val) values))))) c)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-8 is a root stage
+ Stage-6 depends on stages: Stage-1, Stage-8, Stage-9, Stage-10 , 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-11 is a root stage , consists of Stage-12, Stage-13, Stage-1
+ Stage-12 has a backup stage: Stage-1
+ Stage-9 depends on stages: Stage-12
+ Stage-13 has a backup stage: Stage-1
+ Stage-10 depends on stages: Stage-13
+ Stage-1
+
+STAGE PLANS:
+ Stage: Stage-8
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1:c-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToString(_col1)
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Block level merge
+
+ Stage: Stage-4
+ Block level merge
+
+ Stage: Stage-5
+ Move Operator
+ files:
+ hdfs directory: true
+#### A masked pattern was here ####
+
+ Stage: Stage-11
+ Conditional Operator
+
+ Stage: Stage-12
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ null-subquery2:c-subquery2:b
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ null-subquery2:c-subquery2:b
+ TableScan
+ alias: b
+ HashTable Sink Operator
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ Position of Big Table: 0
+
+ Stage: Stage-9
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery2:c-subquery2:a
+ TableScan
+ alias: a
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ outputColumnNames: _col0, _col5
+ Position of Big Table: 0
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Local Work:
+ Map Reduce Local Work
+
+ Stage: Stage-13
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ null-subquery2:c-subquery2:a
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ null-subquery2:c-subquery2:a
+ TableScan
+ alias: a
+ HashTable Sink Operator
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ Position of Big Table: 1
+
+ Stage: Stage-10
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery2:c-subquery2:b
+ TableScan
+ alias: b
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ outputColumnNames: _col0, _col5
+ Position of Big Table: 1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Local Work:
+ Map Reduce Local Work
+
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery2:c-subquery2:a
+ TableScan
+ alias: a
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 0
+ value expressions:
+ expr: key
+ type: string
+ null-subquery2:c-subquery2:b
+ TableScan
+ alias: b
+ Reduce Output Operator
+ key expressions:
+ expr: key
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: key
+ type: string
+ tag: 1
+ value expressions:
+ expr: val
+ type: string
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {VALUE._col0}
+ 1 {VALUE._col1}
+ handleSkewJoin: false
+ outputColumnNames: _col0, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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 a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+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 a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@inputtbl1
+POSTHOOK: Output: default@outputtbl1
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ]
+PREHOOK: query: desc formatted outputTbl1
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), ]
+1 1
+1 11
+2 1
+2 12
+3 1
+3 13
+7 1
+7 17
+8 1
+8 1
+8 18
+8 18
+8 28
+8 28
Index: ql/src/test/results/clientpositive/union_remove_7.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_7.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_7.q.out (working copy)
@@ -0,0 +1,281 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+1 1
+1 1
+2 1
+2 1
+3 1
+3 1
+7 1
+7 1
+8 2
+8 2
Index: ql/src/test/results/clientpositive/union_remove_2.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_2.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_2.q.out (working copy)
@@ -0,0 +1,302 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 1 values))))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR 2 values))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 2
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1-subquery1:a-subquery1-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1-subquery2:a-subquery1-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: 1
+ type: int
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+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
Index: ql/src/test/results/clientpositive/union_remove_20.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_20.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_20.q.out (working copy)
@@ -0,0 +1,279 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(values bigint, key string) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(values bigint, key string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) values)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col1
+ type: bigint
+ expr: _col0
+ type: string
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col1
+ type: bigint
+ expr: _col0
+ type: string
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+values bigint None
+key string None
+
+# 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: 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 ####
+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, ]
+1 1
+1 1
+1 2
+1 2
+1 3
+1 3
+1 7
+1 7
+2 8
+2 8
Index: ql/src/test/results/clientpositive/union_remove_18.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_18.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_18.q.out (working copy)
@@ -0,0 +1,429 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values) (TOK_SELEXPR (TOK_TABLE_OR_COL ds))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (TOK_TABLE_OR_COL ds)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values) (TOK_SELEXPR (TOK_TABLE_OR_COL ds))) (TOK_GROUPBY (TOK_TABLE_OR_COL key) (TOK_TABLE_OR_COL ds))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1) (TOK_PARTSPEC (TOK_PARTVAL ds)))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: ds
+ type: string
+ outputColumnNames: key, ds
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ expr: ds
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col2
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ expr: KEY._col1
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: bigint
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ expr: ds
+ type: string
+ outputColumnNames: key, ds
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ expr: ds
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ sort order: ++
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col2
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ expr: KEY._col1
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col2
+ type: bigint
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ expr: _col2
+ type: string
+ outputColumnNames: _col0, _col1, _col2
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+
+# 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
+POSTHOOK: query: show partitions outputTbl1
+POSTHOOK: type: SHOWPARTITIONS
+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, ]
+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@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@ds=11
+#### A masked pattern was here ####
+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, ]
+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@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@ds=18
+#### A masked pattern was here ####
+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, ]
+8 1 18
+8 1 18
+PREHOOK: query: select * from outputTbl1 where ds is not null order by key, values
+PREHOOK: type: QUERY
+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
+POSTHOOK: type: QUERY
+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 ####
+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, ]
+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 28
+8 1 28
+8 1 18
Index: ql/src/test/results/clientpositive/union_remove_13.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_13.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_13.q.out (working copy)
@@ -0,0 +1,355 @@
+PREHOOK: query: -- This is to test the union->selectstar->filesink optimization
+-- Union of 2 subqueries is performed (one of which is a mapred query, and the
+-- other one is a map-join query), followed by select star and a file sink.
+-- The union selectstar optimization should be performed, and the union should be removed.
+
+-- 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 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
+
+create table inputTbl1(key string, val string) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- This is to test the union->selectstar->filesink optimization
+-- Union of 2 subqueries is performed (one of which is a mapred query, and the
+-- other one is a map-join query), followed by select star and a file sink.
+-- The union selectstar optimization should be performed, and the union should be removed.
+
+-- 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 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
+
+create table inputTbl1(key string, val string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as rcfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+POSTHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME inputTbl1) a) (TOK_TABREF (TOK_TABNAME inputTbl1) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key) key) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) val) values))))) c)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+STAGE DEPENDENCIES:
+ Stage-9 is a root stage
+ Stage-7 depends on stages: Stage-2, Stage-9 , 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-10 is a root stage
+ Stage-1 depends on stages: Stage-10
+ Stage-2 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-9
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery1:c-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToString(_col1)
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Block level merge
+
+ Stage: Stage-5
+ Block level merge
+
+ Stage: Stage-6
+ Move Operator
+ files:
+ hdfs directory: true
+#### A masked pattern was here ####
+
+ Stage: Stage-10
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ null-subquery2:c-subquery2:a
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ null-subquery2:c-subquery2:a
+ TableScan
+ alias: a
+ HashTable Sink Operator
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ Position of Big Table: 1
+
+ Stage: Stage-1
+ Map Reduce
+ Alias -> Map Operator Tree:
+ null-subquery2:c-subquery2:b
+ TableScan
+ alias: b
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ condition expressions:
+ 0 {key}
+ 1 {val}
+ handleSkewJoin: false
+ keys:
+ 0 [Column[key]]
+ 1 [Column[key]]
+ outputColumnNames: _col0, _col5
+ Position of Big Table: 1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ Local Work:
+ Map Reduce Local Work
+
+ Stage: Stage-2
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col5
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col5
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: string
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: UDFToLong(_col1)
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@inputtbl1
+POSTHOOK: Output: default@outputtbl1
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), (inputtbl1)inputtbl1.null, ]
+PREHOOK: query: desc formatted outputTbl1
+PREHOOK: type: DESCTABLE
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), (inputtbl1)inputtbl1.null, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+POSTHOOK: Lineage: outputtbl1.key EXPRESSION [(inputtbl1)a.FieldSchema(name:key, type:string, comment:null), (inputtbl1)inputtbl1.FieldSchema(name:key, type:string, comment:null), ]
+POSTHOOK: Lineage: outputtbl1.values EXPRESSION [(inputtbl1)b.FieldSchema(name:val, type:string, comment:null), (inputtbl1)inputtbl1.null, ]
+1 1
+1 11
+2 1
+2 12
+3 1
+3 13
+7 1
+7 17
+8 2
+8 18
+8 18
+8 28
+8 28
Index: ql/src/test/results/clientpositive/union_remove_6.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_6.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_6.q.out (working copy)
@@ -0,0 +1,337 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: create table outputTbl2(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl2(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl2
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl2))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+ Stage: Stage-3
+ Map Reduce
+ Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+ TableScan
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 2
+ 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
+#### A masked pattern was here ####
+ TableScan
+ Union
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 2
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+
+
+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 ####
+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, ]
+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 ####
+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, ]
+1 1
+1 1
+2 1
+2 1
+3 1
+3 1
+7 1
+7 1
+8 2
+8 2
Index: ql/src/test/results/clientpositive/union_remove_1.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_1.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_1.q.out (working copy)
@@ -0,0 +1,277 @@
+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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key string, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL key)) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: string
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key string None
+values bigint None
+
+# 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: 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 ####
+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, ]
+1 1
+1 1
+2 1
+2 1
+3 1
+3 1
+7 1
+7 1
+8 2
+8 2
Index: ql/src/test/results/clientpositive/union_remove_24.q.out
===================================================================
--- ql/src/test/results/clientpositive/union_remove_24.q.out (revision 0)
+++ ql/src/test/results/clientpositive/union_remove_24.q.out (working copy)
@@ -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.
+-- 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
+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: default@inputTbl1
+PREHOOK: query: create table outputTbl1(key double, values bigint) stored as textfile
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table outputTbl1(key double, values bigint) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@outputTbl1
+PREHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+PREHOOK: type: LOAD
+PREHOOK: Output: default@inputtbl1
+POSTHOOK: query: load data local inpath '../data/files/T1.txt' into table inputTbl1
+POSTHOOK: type: LOAD
+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
+ABSTRACT SYNTAX TREE:
+ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION TOK_DOUBLE (TOK_TABLE_OR_COL key)) key) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME inputTbl1))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION TOK_BIGINT (TOK_TABLE_OR_COL key)) key) (TOK_SELEXPR (TOK_FUNCTION count 1) values)) (TOK_GROUPBY (TOK_TABLE_OR_COL key))))) a)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME outputTbl1))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF))))
+
+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
+ Alias -> Map Operator Tree:
+ null-subquery2:a-subquery2:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: UDFToLong(_col0)
+ type: bigint
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: UDFToDouble(_col0)
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+ Alias -> Map Operator Tree:
+ null-subquery1:a-subquery1:inputtbl1
+ TableScan
+ alias: inputtbl1
+ Select Operator
+ expressions:
+ expr: key
+ type: string
+ outputColumnNames: key
+ Group By Operator
+ aggregations:
+ expr: count(1)
+ bucketGroup: false
+ keys:
+ expr: key
+ type: string
+ mode: hash
+ outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions:
+ expr: _col0
+ type: string
+ sort order: +
+ Map-reduce partition columns:
+ expr: _col0
+ type: string
+ tag: -1
+ value expressions:
+ expr: _col1
+ type: bigint
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations:
+ expr: count(VALUE._col0)
+ bucketGroup: false
+ keys:
+ expr: KEY._col0
+ type: string
+ mode: mergepartial
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: UDFToDouble(_col0)
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ Select Operator
+ expressions:
+ expr: _col0
+ type: double
+ expr: _col1
+ type: bigint
+ outputColumnNames: _col0, _col1
+ File Output Operator
+ compressed: false
+ GlobalTableId: 1
+ 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
+POSTHOOK: query: desc formatted outputTbl1
+POSTHOOK: type: DESCTABLE
+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, ]
+# col_name data_type comment
+
+key double None
+values bigint None
+
+# 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: 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 ####
+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, ]
+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
Index: ql/src/test/queries/clientpositive/union_remove_20.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_20.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_20.q (working copy)
@@ -0,0 +1,45 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_13.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_13.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_13.q (working copy)
@@ -0,0 +1,50 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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 mapred query, and the
+-- other one is a map-join query), followed by select star and a file sink.
+-- The union selectstar optimization should be performed, and the union should be removed.
+
+-- 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 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
+
+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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c;
+
+insert overwrite table outputTbl1
+SELECT * FROM
+(
+select key, count(1) as values from inputTbl1 group by key
+union all
+select /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c;
+
+desc formatted outputTbl1;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+select * from outputTbl1 order by key, values;
Index: ql/src/test/queries/clientpositive/union_remove_8.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_8.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_8.q (working copy)
@@ -0,0 +1,51 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_24.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_24.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_24.q (working copy)
@@ -0,0 +1,43 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_17.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_17.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_17.q (working copy)
@@ -0,0 +1,49 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q
===================================================================
--- ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q (revision 0)
+++ ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q (working copy)
@@ -0,0 +1,40 @@
+set hive.internal.ddl.list.bucketing.enable=true;
+set hive.optimize.skewjoin.compiletime = true;
+set hive.mapred.supports.subdirectories=true;
+
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+
+set hive.merge.mapfiles=false;
+set hive.merge.mapredfiles=false;
+set mapred.input.dir.recursive=true;
+
+CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+
+CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+
+CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+
+-- This is to test the union->selectstar->filesink and skewjoin optimization
+-- Union of 3 map-reduce subqueries is performed for the skew join
+-- 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.
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+-- Since this test creates sub-directories for the output table, it might be easier
+-- to run the test only on hadoop 23
+
+EXPLAIN
+SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+SELECT a.*, b.*, c.* FROM T1 a JOIN T2 b ON a.key = b.key JOIN T3 c on a.key = c.key
+ORDER BY a.key, b.key, c.key, a.val, b.val, c.val;
Index: ql/src/test/queries/clientpositive/union_remove_1.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_1.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_1.q (working copy)
@@ -0,0 +1,44 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_10.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_10.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_10.q (working copy)
@@ -0,0 +1,57 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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;
Index: ql/src/test/queries/clientpositive/union_remove_5.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_5.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_5.q (working copy)
@@ -0,0 +1,51 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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;
Index: ql/src/test/queries/clientpositive/union_remove_21.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_21.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_21.q (working copy)
@@ -0,0 +1,45 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_14.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_14.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_14.q (working copy)
@@ -0,0 +1,52 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+set hive.merge.mapfiles=true;
+set hive.merge.mapredfiles=true;
+set hive.auto.convert.join=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 join, which should be performed as a map-join query at runtime),
+-- followed by select star and a file sink.
+-- The union selectstar optimization should be performed, and the union should be removed.
+
+-- 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 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
+
+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 a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c;
+
+insert overwrite table outputTbl1
+SELECT * FROM
+(
+select key, 1 as values from inputTbl1
+union all
+select a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c;
+
+desc formatted outputTbl1;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+select * from outputTbl1 order by key, values;
Index: ql/src/test/queries/clientpositive/union_remove_9.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_9.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_9.q (working copy)
@@ -0,0 +1,55 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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 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;
Index: ql/src/test/queries/clientpositive/union_remove_18.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_18.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_18.q (working copy)
@@ -0,0 +1,53 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_2.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_2.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_2.q (working copy)
@@ -0,0 +1,50 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
+
Index: ql/src/test/queries/clientpositive/union_remove_11.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_11.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_11.q (working copy)
@@ -0,0 +1,57 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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;
Index: ql/src/test/queries/clientpositive/union_remove_6.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_6.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_6.q (working copy)
@@ -0,0 +1,41 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;;
Index: ql/src/test/queries/clientpositive/union_remove_22.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_22.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_22.q (working copy)
@@ -0,0 +1,63 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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, some columns are repeated. 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 selects columns from
+-- both the sub-qeuries of the union) is pushed above the union.
+
+create table inputTbl1(key string, val string) stored as textfile;
+create table outputTbl1(key string, values bigint, values2 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, 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, 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;
+
+explain
+insert overwrite table outputTbl1
+SELECT a.key, concat(a.values, a.values), concat(a.values, 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, concat(a.values, a.values), concat(a.values, 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;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+select * from outputTbl1 order by key, values;
Index: ql/src/test/queries/clientpositive/union_remove_15.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_15.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_15.q (working copy)
@@ -0,0 +1,53 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_19.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_19.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_19.q (working copy)
@@ -0,0 +1,90 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_3.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_3.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_3.q (working copy)
@@ -0,0 +1,50 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
+
Index: ql/src/test/queries/clientpositive/union_remove_12.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_12.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_12.q (working copy)
@@ -0,0 +1,50 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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 is a map-join query), followed by select star and a file sink.
+-- The union optimization is applied, and the union is removed.
+
+-- 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 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
+
+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 /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c;
+
+insert overwrite table outputTbl1
+SELECT * FROM
+(
+select key, 1 as values from inputTbl1
+union all
+select /*+ mapjoin(a) */ a.key as key, b.val as values
+FROM inputTbl1 a join inputTbl1 b on a.key=b.key
+)c;
+
+desc formatted outputTbl1;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+select * from outputTbl1 order by key, values;
Index: ql/src/test/queries/clientpositive/union_remove_7.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_7.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_7.q (working copy)
@@ -0,0 +1,46 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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;
Index: ql/src/test/queries/clientpositive/union_remove_23.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_23.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_23.q (working copy)
@@ -0,0 +1,47 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=true;
+
+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 of the sub-queries
+-- would have multiple map-reduce jobs.
+-- 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
+ (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key
+ UNION ALL
+ SELECT key, count(1) as values from inputTbl1 group by key
+) subq2;
+
+insert overwrite table outputTbl1
+SELECT *
+FROM (
+ SELECT key, count(1) as values from
+ (SELECT a.key, b.val from inputTbl1 a join inputTbl1 b on a.key=b.key) subq group by key
+ UNION ALL
+ SELECT key, count(1) as values from inputTbl1 group by key
+) subq2;
+
+desc formatted outputTbl1;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+select * from outputTbl1 order by key, values;
Index: ql/src/test/queries/clientpositive/union_remove_16.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_16.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_16.q (working copy)
@@ -0,0 +1,52 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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;
Index: ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q
===================================================================
--- ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q (revision 0)
+++ ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q (working copy)
@@ -0,0 +1,68 @@
+set hive.internal.ddl.list.bucketing.enable=true;
+set hive.optimize.skewjoin.compiletime = true;
+set hive.mapred.supports.subdirectories=true;
+
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+
+set hive.merge.mapfiles=false;
+set hive.merge.mapredfiles=false;
+set mapred.input.dir.recursive=true;
+
+-- This is to test the union->selectstar->filesink and skewjoin optimization
+-- Union of 2 map-reduce subqueries is performed for the skew join
+-- 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.
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+-- Since this test creates sub-directories for the output, it might be easier to run the test
+-- only on hadoop 23
+
+CREATE TABLE T1(key STRING, val STRING)
+SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+
+CREATE TABLE T2(key STRING, val STRING)
+SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+
+-- a simple join query with skew on both the tables on the join key
+
+EXPLAIN
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key
+ORDER BY a.key, b.key, a.val, b.val;
+
+-- test outer joins also
+
+EXPLAIN
+SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key;
+
+SELECT a.*, b.* FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key
+ORDER BY a.key, b.key, a.val, b.val;
+
+create table DEST1(key1 STRING, val1 STRING, key2 STRING, val2 STRING);
+
+EXPLAIN
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key;
+
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a JOIN T2 b ON a.key = b.key;
+
+SELECT * FROM DEST1
+ORDER BY key1, key2, val1, val2;
+
+EXPLAIN
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key;
+
+INSERT OVERWRITE TABLE DEST1
+SELECT * FROM T1 a RIGHT OUTER JOIN T2 b ON a.key = b.key;
+
+SELECT * FROM DEST1
+ORDER BY key1, key2, val1, val2;
Index: ql/src/test/queries/clientpositive/union_remove_4.q
===================================================================
--- ql/src/test/queries/clientpositive/union_remove_4.q (revision 0)
+++ ql/src/test/queries/clientpositive/union_remove_4.q (working copy)
@@ -0,0 +1,45 @@
+set hive.stats.autogather=false;
+set hive.optimize.union.remove=true;
+set hive.mapred.supports.subdirectories=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;
Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcContext.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcContext.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcContext.java (working copy)
@@ -23,6 +23,7 @@
import org.apache.hadoop.hive.ql.exec.UnionOperator;
import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
/**
* UnionProcContext.
@@ -112,12 +113,14 @@
}
return true;
}
-
}
// the subqueries are map-only jobs
private boolean mapOnlySubq;
+ // ParseContext
+ private ParseContext parseContext;
+
/**
* @return the mapOnlySubq
*/
@@ -147,4 +150,12 @@
public UnionParseContext getUnionParseContext(UnionOperator u) {
return uCtxMap.get(u);
}
+
+ public ParseContext getParseContext() {
+ return parseContext;
+ }
+
+ public void setParseContext(ParseContext parseContext) {
+ this.parseContext = parseContext;
+ }
}
Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcFactory.java (working copy)
@@ -17,16 +17,21 @@
*/
package org.apache.hadoop.hive.ql.optimizer.unionproc;
+import java.util.ArrayList;
import java.util.List;
import java.util.Stack;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
import org.apache.hadoop.hive.ql.exec.UnionOperator;
import org.apache.hadoop.hive.ql.lib.Node;
import org.apache.hadoop.hive.ql.lib.NodeProcessor;
import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
import org.apache.hadoop.hive.ql.optimizer.unionproc.UnionProcContext.UnionParseContext;
import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
/**
@@ -146,7 +151,7 @@
UnionOperator parentUnionOperator = null;
while (start >= 0) {
Operator extends OperatorDesc> parent =
- (Operator extends OperatorDesc>) stack.get(start);
+ (Operator extends OperatorDesc>) stack.get(start);
if (parent instanceof UnionOperator) {
parentUnionOperator = (UnionOperator) parent;
break;
@@ -158,7 +163,7 @@
boolean mapOnly = false;
if (parentUnionOperator != null) {
UnionParseContext parentUCtx =
- ctx.getUnionParseContext(parentUnionOperator);
+ ctx.getUnionParseContext(parentUnionOperator);
if (parentUCtx != null && parentUCtx.allMapOnlySubQSet()) {
mapOnly = parentUCtx.allMapOnlySubQ();
}
@@ -173,6 +178,139 @@
}
/**
+ * Union followed by no processing.
+ * This is to optimize queries of the type:
+ * select * from (subq1 union all subq2 ...)x;
+ * where at least one of the queries involve a map-reduce job.
+ * There is no need for a union in this scenario - it involves an extra
+ * write and read for the final output without this optimization.
+ * Queries of the form:
+ * select x.c1 from (subq1 union all subq2 ...)x where filter(x.c2);
+ * can be transformed to:
+ * select * from (subq1 where filter union all subq2 where filter ...)x;
+ * and then optimized.
+ */
+ public static class UnionNoProcessFile implements NodeProcessor {
+
+ private void pushOperatorsAboveUnion(UnionOperator union,
+ Stack stack, int pos) throws SemanticException {
+ // Clone all the operators between union and filescan, and push them above
+ // the union. Remove the union (the tree below union gets delinked after that)
+ try {
+ List> parents =
+ union.getParentOperators();
+ int numParents = parents.size();
+ for (Operator extends OperatorDesc> parent : parents) {
+ parent.setChildOperators(null);
+ }
+
+ for (; pos < stack.size() - 1; pos++) {
+ Operator extends OperatorDesc> originalOp =
+ (Operator extends OperatorDesc>)stack.get(pos);
+
+ for (int p = 0; p < numParents; p++) {
+ OperatorDesc cloneDesc = (OperatorDesc)originalOp.getConf().clone();
+
+ Operator extends OperatorDesc> cloneOp =
+ OperatorFactory.getAndMakeChild(cloneDesc, originalOp.getSchema(), parents.get(p));
+ parents.set(p, cloneOp);
+ }
+ }
+
+ // FileSink cannot be simply cloned - it requires some special processing.
+ // Sub-queries for the union will be processed as independent map-reduce jobs
+ // possibly running in parallel. Those sub-queries cannot write to the same
+ // directory. Clone the filesink, but create a sub-directory in the final path
+ // for each sub-query. Also, these different filesinks need to be linked to each other
+ FileSinkOperator fileSinkOp = (FileSinkOperator)stack.get(pos);
+ // For file sink operator, change the directory name
+ String parentDirName = fileSinkOp.getConf().getDirName();
+
+ // Clone the fileSinkDesc of the final fileSink and create similar fileSinks at
+ // each parent
+ List fileDescLists = new ArrayList();
+
+ for (Operator extends OperatorDesc> parent : parents) {
+ FileSinkDesc fileSinkDesc = (FileSinkDesc) fileSinkOp.getConf().clone();
+
+ String dirName = parentDirName + Path.SEPARATOR + parent.getIdentifier() ;
+ fileSinkDesc.setDirName(dirName);
+ fileSinkDesc.setLinkedFileSink(true);
+ fileSinkDesc.setParentDir(parentDirName);
+ parent.setChildOperators(null);
+ Operator extends OperatorDesc> tmpFileSinkOp =
+ OperatorFactory.getAndMakeChild(fileSinkDesc, parent.getSchema(), parent);
+ tmpFileSinkOp.setChildOperators(null);
+ fileDescLists.add(fileSinkDesc);
+ }
+
+ for (FileSinkDesc fileDesc : fileDescLists) {
+ fileDesc.setLinkedFileSinkDesc(fileDescLists);
+ }
+
+ // delink union
+ union.setChildOperators(null);
+ union.setParentOperators(null);
+ } catch (Exception e) {
+ throw new SemanticException(e.getMessage());
+ }
+ }
+
+ @Override
+ public Object process(Node nd, Stack stack, NodeProcessorCtx procCtx,
+ Object... nodeOutputs) throws SemanticException {
+ FileSinkOperator fileSinkOp = (FileSinkOperator)nd;
+
+ // Has this filesink already been processed
+ if (fileSinkOp.getConf().isLinkedFileSink()) {
+ return null;
+ }
+
+ int size = stack.size();
+ int pos = size - 2;
+ UnionOperator union = null;
+
+ // Walk the tree. As long as the operators between the union and the filesink
+ // do not involve a reducer, and they can be pushed above the union, it makes
+ // sense to push them above the union, and remove the union. An interface
+ // has been added to the operator 'supportUnionRemoveOptimization' to denote whether
+ // this operator can be removed.
+ while (pos >= 0) {
+ Operator extends OperatorDesc> operator =
+ (Operator extends OperatorDesc>)stack.get(pos);
+
+ // Break if it encountered a union
+ if (operator instanceof UnionOperator) {
+ union = (UnionOperator)operator;
+ // No need for this optimization in case of multi-table inserts
+ if (union.getChildOperators().size() > 1) {
+ return null;
+ }
+ break;
+ }
+
+ if (!operator.supportUnionRemoveOptimization()) {
+ return null;
+ }
+ pos--;
+ }
+
+ UnionProcContext ctx = (UnionProcContext) procCtx;
+ UnionParseContext uCtx = ctx.getUnionParseContext(union);
+
+ // No need for this if all sub-queries are map-only queries
+ // If all the queries are map-only, anyway the query is most optimized
+ if ((uCtx != null) && (uCtx.allMapOnlySubQ())) {
+ return null;
+ }
+
+ pos++;
+ pushOperatorsAboveUnion(union, stack, pos);
+ return null;
+ }
+ }
+
+ /**
* Default processor.
*/
public static class NoUnion implements NodeProcessor {
@@ -204,4 +342,7 @@
return new NoUnion();
}
+ public static NodeProcessor getUnionNoProcessFile() {
+ return new UnionNoProcessFile();
+ }
}
Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/unionproc/UnionProcessor.java (working copy)
@@ -22,11 +22,13 @@
import java.util.LinkedHashMap;
import java.util.Map;
-import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
import org.apache.hadoop.hive.ql.exec.TableScanOperator;
import org.apache.hadoop.hive.ql.exec.UnionOperator;
-import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
import org.apache.hadoop.hive.ql.lib.Dispatcher;
import org.apache.hadoop.hive.ql.lib.GraphWalker;
@@ -77,13 +79,14 @@
opRules.put(new RuleRegExp("R3",
TableScanOperator.getOperatorName() + "%.*" + UnionOperator.getOperatorName() + "%"),
UnionProcFactory.getMapUnion());
- opRules.put(new RuleRegExp("R3",
+ opRules.put(new RuleRegExp("R4",
MapJoinOperator.getOperatorName() + "%.*" + UnionOperator.getOperatorName() + "%"),
UnionProcFactory.getMapJoinUnion());
// The dispatcher fires the processor for the matching rule and passes the
// context along
UnionProcContext uCtx = new UnionProcContext();
+ uCtx.setParseContext(pCtx);
Dispatcher disp = new DefaultRuleDispatcher(UnionProcFactory.getNoUnion(),
opRules, uCtx);
GraphWalker ogw = new PreOrderWalker(disp);
@@ -94,6 +97,29 @@
ogw.startWalking(topNodes, null);
pCtx.setUCtx(uCtx);
+ // Walk the tree again to see if the union can be removed completely
+ HiveConf conf = pCtx.getConf();
+ opRules.clear();
+ if (conf.getBoolVar(HiveConf.ConfVars.HIVE_OPTIMIZE_UNION_REMOVE)) {
+
+ if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES)) {
+ throw new
+ SemanticException(ErrorMsg.HIVE_UNION_REMOVE_OPTIMIZATION_NEEDS_SUBDIRECTORIES.getMsg());
+ }
+
+ opRules.put(new RuleRegExp("R5", UnionOperator.getOperatorName() + "%" +
+ ".*" + FileSinkOperator.getOperatorName() + "%"),
+ UnionProcFactory.getUnionNoProcessFile());
+
+ disp = new DefaultRuleDispatcher(UnionProcFactory.getNoUnion(), opRules, uCtx);
+ ogw = new PreOrderWalker(disp);
+
+ // Create a list of topop nodes
+ topNodes.clear();
+ topNodes.addAll(pCtx.getTopOps().values());
+ ogw.startWalking(topNodes, null);
+ }
+
return pCtx;
}
}
Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRProcContext.java (working copy)
@@ -23,6 +23,7 @@
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import org.apache.hadoop.hive.conf.HiveConf;
@@ -38,6 +39,7 @@
import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
import org.apache.hadoop.hive.ql.parse.ParseContext;
import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
import org.apache.hadoop.hive.ql.plan.MoveWork;
import org.apache.hadoop.hive.ql.plan.OperatorDesc;
@@ -253,6 +255,10 @@
private List> rootOps;
private DependencyCollectionTask dependencyTaskForMultiInsert;
+ // 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;
+
/**
* Set of read entities. This list is generated by the walker and is passed to
* the hooks.
@@ -314,6 +320,7 @@
unionTaskMap = new HashMap();
mapJoinTaskMap = new HashMap, GenMRMapJoinCtx>();
dependencyTaskForMultiInsert = null;
+ linkedFileDescTasks = null;
}
/**
@@ -568,4 +575,13 @@
}
return dependencyTaskForMultiInsert;
}
+
+ public Map> getLinkedFileDescTasks() {
+ return linkedFileDescTasks;
+ }
+
+ public void setLinkedFileDescTasks(
+ Map> linkedFileDescTasks) {
+ this.linkedFileDescTasks = linkedFileDescTasks;
+ }
}
Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java (working copy)
@@ -23,6 +23,7 @@
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
+import java.util.Map;
import java.util.Stack;
import org.apache.commons.logging.Log;
@@ -109,6 +110,14 @@
parseCtx.getQB().getParseInfo().isInsertToTable();
HiveConf hconf = parseCtx.getConf();
+ // If this file sink desc has been processed due to a linked file sink desc,
+ // use that task
+ Map> fileSinkDescs = ctx.getLinkedFileDescTasks();
+ if (fileSinkDescs != null) {
+ Task extends Serializable> childTask = fileSinkDescs.get(fsOp.getConf());
+ processLinkedFileDesc(ctx, childTask);
+ return null;
+ }
// Has the user enabled merging of files for map-only jobs or for all jobs
if ((ctx.getMvTask() != null) && (!ctx.getMvTask().isEmpty())) {
@@ -129,19 +138,31 @@
}
if ((mvTask != null) && !mvTask.isLocal()) {
- // There are separate configuration parameters to control whether to
- // merge for a map-only job
- // or for a map-reduce job
- MapredWork currWork = (MapredWork) currTask.getWork();
- boolean mergeMapOnly =
- hconf.getBoolVar(HiveConf.ConfVars.HIVEMERGEMAPFILES) &&
- currWork.getReducer() == null;
- boolean mergeMapRed =
- hconf.getBoolVar(HiveConf.ConfVars.HIVEMERGEMAPREDFILES) &&
- currWork.getReducer() != null;
- if (mergeMapOnly || mergeMapRed) {
- chDir = true;
+ if (fsOp.getConf().isLinkedFileSink()) {
+ // If the user has HIVEMERGEMAPREDFILES set to false, the idea was the
+ // number of reducers are few, so the number of files anyway are small.
+ // However, with this optimization, we are increasing the number of files
+ // possibly by a big margin. So, merge aggresively.
+ if (hconf.getBoolVar(HiveConf.ConfVars.HIVEMERGEMAPFILES) ||
+ hconf.getBoolVar(HiveConf.ConfVars.HIVEMERGEMAPREDFILES)) {
+ chDir = true;
+ }
}
+ else {
+ // There are separate configuration parameters to control whether to
+ // merge for a map-only job
+ // or for a map-reduce job
+ MapredWork currWork = (MapredWork) currTask.getWork();
+ boolean mergeMapOnly =
+ hconf.getBoolVar(HiveConf.ConfVars.HIVEMERGEMAPFILES) &&
+ currWork.getReducer() == null;
+ boolean mergeMapRed =
+ hconf.getBoolVar(HiveConf.ConfVars.HIVEMERGEMAPREDFILES) &&
+ currWork.getReducer() != null;
+ if (mergeMapOnly || mergeMapRed) {
+ chDir = true;
+ }
+ }
}
}
}
@@ -153,9 +174,57 @@
createMergeJob((FileSinkOperator) nd, ctx, finalName);
}
+ FileSinkDesc fileSinkDesc = fsOp.getConf();
+ if (fileSinkDesc.isLinkedFileSink()) {
+ Map> linkedFileDescTasks =
+ ctx.getLinkedFileDescTasks();
+ if (linkedFileDescTasks == null) {
+ linkedFileDescTasks = new HashMap>();
+ ctx.setLinkedFileDescTasks(linkedFileDescTasks);
+ }
+
+ // The child tasks may be null in case of a select
+ if ((currTask.getChildTasks() != null) &&
+ (currTask.getChildTasks().size() == 1)) {
+ for (FileSinkDesc fileDesc : fileSinkDesc.getLinkedFileSinkDesc()) {
+ linkedFileDescTasks.put(fileDesc, currTask.getChildTasks().get(0));
+ }
+ }
+ }
+
return null;
}
+ /*
+ * Multiple file sink descriptors are linked.
+ * Use the task created by the first linked file descriptor
+ */
+ private void processLinkedFileDesc(GenMRProcContext ctx,
+ Task extends Serializable> childTask)
+ throws SemanticException {
+ Operator extends OperatorDesc> currTopOp = ctx.getCurrTopOp();
+ String currAliasId = ctx.getCurrAliasId();
+ List> seenOps = ctx.getSeenOps();
+ List> rootTasks = ctx.getRootTasks();
+ Task extends Serializable> currTask = ctx.getCurrTask();
+
+ if (currTopOp != null) {
+ if (!seenOps.contains(currTopOp)) {
+ seenOps.add(currTopOp);
+ GenMapRedUtils.setTaskPlan(currAliasId, currTopOp,
+ (MapredWork) currTask.getWork(), false, ctx);
+ }
+
+ if (!rootTasks.contains(currTask)) {
+ rootTasks.add(currTask);
+ }
+ }
+
+ if (childTask != null) {
+ currTask.addDependentTask(childTask);
+ }
+ }
+
/**
* Add the StatsTask as a dependent task of the MoveTask
* because StatsTask will change the Table/Partition metadata. For atomicity, we
@@ -262,10 +331,10 @@
// since it is unknown if the merge MR will be triggered at execution time.
MoveWork dummyMv = new MoveWork(null, null, null,
- new LoadFileDesc(fsConf.getDirName(), finalName, true, null, null), false);
+ new LoadFileDesc(fsConf.getFinalDirName(), finalName, true, null, null), false);
ConditionalTask cndTsk = createCondTask(conf, currTask, dummyMv, cplan,
- fsConf.getDirName());
+ fsConf.getFinalDirName());
linkMoveTask(ctx, newOutput, cndTsk);
}
@@ -387,7 +456,7 @@
//
MapRedTask currTask = (MapRedTask) ctx.getCurrTask();
MoveWork dummyMv = new MoveWork(null, null, null,
- new LoadFileDesc(fsInputDesc.getDirName(), finalName, true, null, null), false);
+ new LoadFileDesc(fsInputDesc.getFinalDirName(), finalName, true, null, null), false);
MapredWork cplan;
if(parseCtx.getConf().getBoolVar(HiveConf.ConfVars.
@@ -417,7 +486,7 @@
// NOTE: we should gather stats in MR1 rather than MR2 at merge job since we don't
// know if merge MR2 will be triggered at execution time
ConditionalTask cndTsk = createCondTask(ctx.getConf(), ctx.getCurrTask(), dummyMv, cplan,
- fsInputDesc.getDirName());
+ fsInputDesc.getFinalDirName());
// keep the dynamic partition context in conditional task resolver context
ConditionalResolverMergeFilesCtx mrCtx =
@@ -516,7 +585,7 @@
FileSinkDesc fsDesc) {
ArrayList aliases = new ArrayList();
- String inputDir = fsDesc.getDirName();
+ String inputDir = fsDesc.getFinalDirName();
TableDesc tblDesc = fsDesc.getTableInfo();
aliases.add(inputDir); // dummy alias: just use the input path
@@ -541,7 +610,7 @@
private MapredWork createRCFileMergeTask(FileSinkDesc fsInputDesc,
String finalName, boolean hasDynamicPartitions) throws SemanticException {
- String inputDir = fsInputDesc.getDirName();
+ String inputDir = fsInputDesc.getFinalDirName();
TableDesc tblDesc = fsInputDesc.getTableInfo();
if(tblDesc.getInputFileFormatClass().equals(RCFileInputFormat.class)) {
@@ -637,8 +706,9 @@
srcDir = mvWork.getLoadTableWork().getSourceDir();
}
+ String fsOpDirName = fsOp.getConf().getFinalDirName();
if ((srcDir != null)
- && (srcDir.equalsIgnoreCase(fsOp.getConf().getDirName()))) {
+ && (srcDir.equalsIgnoreCase(fsOpDirName))) {
return mvTsk;
}
}
@@ -681,7 +751,7 @@
String dest = null;
if (chDir) {
- dest = fsOp.getConf().getDirName();
+ dest = fsOp.getConf().getFinalDirName();
// generate the temporary file
// it must be on the same file system as the current destination
@@ -689,7 +759,17 @@
Context baseCtx = parseCtx.getContext();
String tmpDir = baseCtx.getExternalTmpFileURI((new Path(dest)).toUri());
- fsOp.getConf().setDirName(tmpDir);
+ FileSinkDesc fileSinkDesc = fsOp.getConf();
+ // Change all the linked file sink descriptors
+ if (fileSinkDesc.isLinkedFileSink()) {
+ for (FileSinkDesc fsConf:fileSinkDesc.getLinkedFileSinkDesc()) {
+ String fileName = Utilities.getFileNameFromDirName(fsConf.getDirName());
+ fsConf.setParentDir(tmpDir);
+ fsConf.setDirName(tmpDir + Path.SEPARATOR + fileName);
+ }
+ } else {
+ fileSinkDesc.setDirName(tmpDir);
+ }
}
Task mvTask = null;
@@ -707,7 +787,6 @@
// Set the move task to be dependent on the current task
if (mvTask != null) {
-
addDependentMoveTasks(ctx, mvTask, currTask);
}
@@ -742,7 +821,7 @@
}
}
// mapTask and currTask should be merged by and join/union operator
- // (e.g., GenMRUnion1j) which has multiple topOps.
+ // (e.g., GenMRUnion1) which has multiple topOps.
// assert mapTask == currTask : "mapTask.id = " + mapTask.getId()
// + "; currTask.id = " + currTask.getId();
}
Index: ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/metadata/Table.java (working copy)
@@ -622,7 +622,7 @@
FileSystem fs;
try {
fs = FileSystem.get(getDataLocation(), Hive.get().getConf());
- Hive.copyFiles(srcf, new Path(getDataLocation().getPath()), fs);
+ Hive.copyFiles(Hive.get().getConf(), srcf, new Path(getDataLocation().getPath()), fs);
} catch (IOException e) {
throw new HiveException("addFiles: filesystem error in check phase", e);
}
Index: ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (working copy)
@@ -31,17 +31,16 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
-import java.util.HashSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FsShell;
@@ -63,7 +62,6 @@
import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
import org.apache.hadoop.hive.metastore.api.HiveObjectType;
import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
@@ -1181,7 +1179,7 @@
Hive.replaceFiles(loadPath, newPartPath, oldPartPath, getConf());
} else {
FileSystem fs = FileSystem.get(tbl.getDataLocation(), getConf());
- Hive.copyFiles(loadPath, newPartPath, fs);
+ Hive.copyFiles(conf, loadPath, newPartPath, fs);
}
// recreate the partition if it existed before
@@ -1225,7 +1223,9 @@
FileStatus[] leafStatus = Utilities.getFileStatusRecurse(loadPath, numDP+1, fs);
// Check for empty partitions
for (FileStatus s : leafStatus) {
- if (s.isDir()) {
+ // Check if the hadoop version supports sub-directories for tables/partitions
+ if (s.isDir() &&
+ !conf.getBoolVar(HiveConf.ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES)) {
// No leaves in this directory
LOG.info("NOT moving empty directory: " + s.getPath());
} else {
@@ -1884,8 +1884,9 @@
}
}
- static private void checkPaths(FileSystem fs, FileStatus[] srcs, Path destf,
- boolean replace) throws HiveException {
+ static private void checkPaths(HiveConf conf,
+ FileSystem fs, FileStatus[] srcs, Path destf,
+ boolean replace) throws HiveException {
try {
for (FileStatus src : srcs) {
FileStatus[] items = fs.listStatus(src.getPath());
@@ -1899,7 +1900,9 @@
fs.delete(itemStaging, true);
continue;
}
- if (item.isDir()) {
+
+ if (!conf.getBoolVar(HiveConf.ConfVars.HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES) &&
+ item.isDir()) {
throw new HiveException("checkPaths: " + src.getPath()
+ " has nested directory" + itemStaging);
}
@@ -1959,7 +1962,7 @@
}
}
- static protected void copyFiles(Path srcf, Path destf, FileSystem fs)
+ static protected void copyFiles(HiveConf conf, Path srcf, Path destf, FileSystem fs)
throws HiveException {
try {
// create the destination if it does not exist
@@ -1985,7 +1988,7 @@
// srcs = new FileStatus[0]; Why is this needed?
}
// check that source and target paths exist
- checkPaths(fs, srcs, destf, false);
+ checkPaths(conf, fs, srcs, destf, false);
// move it, move it
try {
@@ -2020,7 +2023,7 @@
* The directory where the old data location, need to be cleaned up.
*/
static protected void replaceFiles(Path srcf, Path destf, Path oldPath,
- Configuration conf) throws HiveException {
+ HiveConf conf) throws HiveException {
try {
FileSystem fs = srcf.getFileSystem(conf);
@@ -2036,7 +2039,7 @@
LOG.info("No sources specified to move: " + srcf);
return;
}
- checkPaths(fs, srcs, destf, true);
+ checkPaths(conf, fs, srcs, destf, true);
// point of no return -- delete oldPath
if (oldPath != null) {
Index: ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/ColumnInfo.java (working copy)
@@ -180,4 +180,27 @@
public void setSkewedCol(boolean isSkewedCol) {
this.isSkewedCol = isSkewedCol;
}
+
+ private boolean checkEquals(Object obj1, Object obj2) {
+ return obj1 == null ? obj2 == null : obj1.equals(obj2);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof ColumnInfo) || (obj == null)) {
+ return false;
+ }
+
+ ColumnInfo dest = (ColumnInfo)obj;
+ if ((!checkEquals(internalName, dest.getInternalName())) ||
+ (!checkEquals(alias, dest.getAlias())) ||
+ (!checkEquals(getType(), dest.getType())) ||
+ (isSkewedCol != dest.isSkewedCol()) ||
+ (isVirtualCol != dest.getIsVirtualCol()) ||
+ (isHiddenVirtualCol != dest.isHiddenVirtualCol())) {
+ return false;
+ }
+
+ return true;
+ }
}
Index: ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
===================================================================
--- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (revision 1403193)
+++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (working copy)
@@ -77,6 +77,7 @@
protected transient Path parent;
protected transient HiveOutputFormat, ?> hiveOutputFormat;
protected transient Path specPath;
+ protected transient String childSpecPathDynLinkedPartitions;
protected transient int dpStartCol; // start column # for DP columns
protected transient List dpVals; // array of values corresponding to DP columns
protected transient List