Index: ql/src/test/results/clientpositive/join29.q.out =================================================================== --- ql/src/test/results/clientpositive/join29.q.out (revision 1445446) +++ ql/src/test/results/clientpositive/join29.q.out (working copy) @@ -24,14 +24,10 @@ STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-7 depends on stages: Stage-1, Stage-4 , consists of Stage-8, Stage-9, Stage-2 - Stage-8 has a backup stage: Stage-2 - Stage-5 depends on stages: Stage-8 - Stage-0 depends on stages: Stage-2, Stage-5, Stage-6 + Stage-6 depends on stages: Stage-1, Stage-4 + Stage-5 depends on stages: Stage-6 + Stage-0 depends on stages: Stage-5 Stage-3 depends on stages: Stage-0 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 - Stage-2 Stage-4 is a root stage STAGE PLANS: @@ -91,17 +87,14 @@ input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - Stage: Stage-7 - Conditional Operator - - Stage: Stage-8 + Stage: Stage-6 Map Reduce Local Work Alias -> Map Local Tables: - $INTNAME + $INTNAME1 Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - $INTNAME + $INTNAME1 HashTable Sink Operator condition expressions: 0 {_col0} {_col1} @@ -110,12 +103,12 @@ keys: 0 [Column[_col0]] 1 [Column[_col0]] - Position of Big Table: 0 + Position of Big Table: 1 Stage: Stage-5 Map Reduce Alias -> Map Operator Tree: - $INTNAME1 + $INTNAME Map Join Operator condition map: Inner Join 0 to 1 @@ -127,7 +120,7 @@ 0 [Column[_col0]] 1 [Column[_col0]] outputColumnNames: _col0, _col1, _col3 - Position of Big Table: 0 + Position of Big Table: 1 Select Operator expressions: expr: _col0 @@ -170,136 +163,6 @@ Stage: Stage-3 Stats-Aggr Operator - Stage: Stage-9 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - HashTable Sink Operator - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - - Stage: Stage-6 - Map Reduce - Alias -> Map Operator Tree: - $INTNAME - Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 {_col0} {_col1} - 1 {_col1} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - outputColumnNames: _col0, _col1, _col3 - Position of Big Table: 1 - Select Operator - expressions: - expr: _col0 - type: string - expr: _col1 - type: bigint - expr: _col3 - type: bigint - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: string - expr: UDFToInteger(_col1) - type: int - expr: UDFToInteger(_col2) - type: int - 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.dest_j1 - Local Work: - Map Reduce Local Work - - Stage: Stage-2 - Map Reduce - Alias -> Map Operator Tree: - $INTNAME - 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 - $INTNAME1 - Reduce Output Operator - key expressions: - expr: _col0 - type: string - sort order: + - Map-reduce partition columns: - expr: _col0 - type: string - tag: 0 - value expressions: - expr: _col0 - type: string - expr: _col1 - type: bigint - Reduce Operator Tree: - Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 {VALUE._col0} {VALUE._col1} - 1 {VALUE._col1} - handleSkewJoin: false - outputColumnNames: _col0, _col1, _col3 - Select Operator - expressions: - expr: _col0 - type: string - expr: _col1 - type: bigint - expr: _col3 - type: bigint - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: string - expr: UDFToInteger(_col1) - type: int - expr: UDFToInteger(_col2) - type: int - 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.dest_j1 - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: @@ -356,7 +219,6 @@ input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - PREHOOK: query: INSERT OVERWRITE TABLE dest_j1 SELECT subq1.key, subq1.cnt, subq2.cnt FROM (select x.key, count(1) as cnt from src1 x group by x.key) subq1 JOIN Index: ql/src/test/results/clientpositive/join35.q.out =================================================================== --- ql/src/test/results/clientpositive/join35.q.out (revision 1445446) +++ ql/src/test/results/clientpositive/join35.q.out (working copy) @@ -32,14 +32,10 @@ STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-8 depends on stages: Stage-1, Stage-4 , consists of Stage-9, Stage-10, Stage-2 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 - Stage-0 depends on stages: Stage-2, Stage-6, Stage-7 + Stage-7 depends on stages: Stage-1, Stage-4 + Stage-6 depends on stages: Stage-7 + Stage-0 depends on stages: Stage-6 Stage-3 depends on stages: Stage-0 - Stage-10 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-10 - Stage-2 Stage-4 is a root stage STAGE PLANS: @@ -162,10 +158,7 @@ Truncated Path -> Alias: /src [null-subquery1:subq1-subquery1:x] - Stage: Stage-8 - Conditional Operator - - Stage: Stage-9 + Stage: Stage-7 Map Reduce Local Work Alias -> Map Local Tables: x @@ -414,373 +407,6 @@ Stats-Aggr Operator #### A masked pattern was here #### - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: -#### A masked pattern was here #### - Fetch Operator - limit: -1 -#### A masked pattern was here #### - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: -#### A masked pattern was here #### - TableScan - GatherStats: false - Union - HashTable Sink Operator - condition expressions: - 0 {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 1 -#### A masked pattern was here #### - TableScan - GatherStats: false - Union - HashTable Sink Operator - condition expressions: - 0 {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - Position of Big Table: 1 - - Stage: Stage-7 - Map Reduce - Alias -> Map Operator Tree: - x - TableScan - alias: x - GatherStats: false - Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 {_col1} - 1 {key} {value} - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[key]] - outputColumnNames: _col1, _col2, _col3 - Position of Big Table: 1 - Select Operator - expressions: - expr: _col2 - type: string - expr: _col3 - type: string - expr: _col1 - type: bigint - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: string - expr: _col1 - type: string - expr: UDFToInteger(_col2) - type: int - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 1 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 -#### A masked pattern was here #### - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value,val2 - columns.types string:string:int -#### A masked pattern was here #### - name default.dest_j1 - serialization.ddl struct dest_j1 { string key, string value, i32 val2} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.dest_j1 - TotalFiles: 1 - GatherStats: true - MultiFileSpray: false - Local Work: - Map Reduce Local Work - Needs Tagging: true - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: -mr-10002 - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ - - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ -#### A masked pattern was here #### - Partition - base file name: -mr-10003 - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ - - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ -#### A masked pattern was here #### - Partition - base file name: src1 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.src1 - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct src1 { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 216 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.src1 - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct src1 { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 216 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.src1 - name: default.src1 - Truncated Path -> Alias: - /src1 [x] - - Stage: Stage-2 - Map Reduce - Alias -> Map Operator Tree: -#### A masked pattern was here #### - TableScan - GatherStats: false - Union - Reduce Output Operator - key expressions: - expr: _col0 - type: string - sort order: + - Map-reduce partition columns: - expr: _col0 - type: string - tag: 0 - value expressions: - expr: _col1 - type: bigint -#### A masked pattern was here #### - TableScan - GatherStats: false - Union - Reduce Output Operator - key expressions: - expr: _col0 - type: string - sort order: + - Map-reduce partition columns: - expr: _col0 - type: string - tag: 0 - value expressions: - expr: _col1 - type: bigint - x - TableScan - alias: x - GatherStats: false - 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: value - type: string - Needs Tagging: true - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: -mr-10002 - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ - - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ -#### A masked pattern was here #### - Partition - base file name: -mr-10003 - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ - - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1 - columns.types string,bigint - escape.delim \ -#### A masked pattern was here #### - Partition - base file name: src1 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.src1 - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct src1 { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 216 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value - columns.types string:string -#### A masked pattern was here #### - name default.src1 - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct src1 { string key, string value} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 216 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.src1 - name: default.src1 - Reduce Operator Tree: - Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 {VALUE._col1} - 1 {VALUE._col0} {VALUE._col1} - handleSkewJoin: false - outputColumnNames: _col1, _col2, _col3 - Select Operator - expressions: - expr: _col2 - type: string - expr: _col3 - type: string - expr: _col1 - type: bigint - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: string - expr: _col1 - type: string - expr: UDFToInteger(_col2) - type: int - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 1 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 -#### A masked pattern was here #### - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns key,value,val2 - columns.types string:string:int -#### A masked pattern was here #### - name default.dest_j1 - serialization.ddl struct dest_j1 { string key, string value, i32 val2} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.dest_j1 - TotalFiles: 1 - GatherStats: true - MultiFileSpray: false - Truncated Path -> Alias: - /src1 [x] -#### A masked pattern was here #### - Stage: Stage-4 Map Reduce Alias -> Map Operator Tree: Index: ql/src/test/results/clientpositive/union22.q.out =================================================================== --- ql/src/test/results/clientpositive/union22.q.out (revision 1445446) +++ ql/src/test/results/clientpositive/union22.q.out (working copy) @@ -86,25 +86,47 @@ (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME dst_union22_delta))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_TABLE_OR_COL k1) k1) (TOK_SELEXPR (TOK_TABLE_OR_COL k2) k2) (TOK_SELEXPR (TOK_TABLE_OR_COL k3) k3) (TOK_SELEXPR (TOK_TABLE_OR_COL k4) k4)) (TOK_WHERE (and (= (TOK_TABLE_OR_COL ds) '1') (<= (TOK_TABLE_OR_COL k0) 50))))) (TOK_QUERY (TOK_FROM (TOK_LEFTOUTERJOIN (TOK_TABREF (TOK_TABNAME dst_union22) a) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME dst_union22_delta))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_WHERE (and (= (TOK_TABLE_OR_COL ds) '1') (> (TOK_TABLE_OR_COL k0) 50))))) b) (and (= (. (TOK_TABLE_OR_COL a) k1) (. (TOK_TABLE_OR_COL b) k1)) (= (. (TOK_TABLE_OR_COL a) ds) '1')))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) k1) k1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) k2) k2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) k3) k3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) k4) k4)) (TOK_WHERE (> (. (TOK_TABLE_OR_COL a) k1) 20))))) subq)) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dst_union22) (TOK_PARTSPEC (TOK_PARTVAL ds '2')))) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)))) STAGE DEPENDENCIES: - Stage-7 is a root stage , consists of Stage-8, Stage-1 - Stage-8 has a backup stage: Stage-1 - Stage-6 depends on stages: Stage-8 - Stage-2 depends on stages: Stage-1, Stage-6 + Stage-7 is a root stage + Stage-6 depends on stages: Stage-7 + Stage-2 depends on stages: Stage-6 Stage-0 depends on stages: Stage-2 Stage-3 depends on stages: Stage-0 - Stage-1 STAGE PLANS: Stage: Stage-7 - Conditional Operator - - Stage: Stage-8 Map Reduce Local Work Alias -> Map Local Tables: - null-subquery2:subq-subquery2:b:dst_union22_delta + null-subquery2:subq-subquery2:a Fetch Operator limit: -1 Alias -> Map Local Operator Tree: + null-subquery2:subq-subquery2:a + TableScan + alias: a + GatherStats: false + Filter Operator + isSamplingPred: false + predicate: + expr: (k1 > 20.0) + type: boolean + HashTable Sink Operator + condition expressions: + 0 {k1} {k2} + 1 {_col3} {_col4} + filter mappings: + 0 [1, 1] + filter predicates: + 0 {(ds = '1')} + 1 + handleSkewJoin: false + keys: + 0 [Column[k1]] + 1 [Column[_col1]] + Position of Big Table: 1 + + Stage: Stage-6 + Map Reduce + Alias -> Map Operator Tree: null-subquery2:subq-subquery2:b:dst_union22_delta TableScan alias: dst_union22_delta @@ -123,7 +145,9 @@ expr: k4 type: string outputColumnNames: _col1, _col3, _col4 - HashTable Sink Operator + Map Join Operator + condition map: + Left Outer Join0 to 1 condition expressions: 0 {k1} {k2} 1 {_col3} {_col4} @@ -136,63 +160,34 @@ keys: 0 [Column[k1]] 1 [Column[_col1]] - Position of Big Table: 0 - - Stage: Stage-6 - Map Reduce - Alias -> Map Operator Tree: - null-subquery2:subq-subquery2:a - TableScan - alias: a - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: - expr: (k1 > 20.0) - type: boolean - Map Join Operator - condition map: - Left Outer Join0 to 1 - condition expressions: - 0 {k1} {k2} - 1 {_col3} {_col4} - filter mappings: - 0 [1, 1] - filter predicates: - 0 {(ds = '1')} - 1 - handleSkewJoin: false - keys: - 0 [Column[k1]] - 1 [Column[_col1]] - outputColumnNames: _col0, _col1, _col10, _col11 - Position of Big Table: 0 - Select Operator - expressions: - expr: _col0 - type: string - expr: _col1 - type: string - expr: _col10 - type: string - expr: _col11 - type: string - outputColumnNames: _col0, _col1, _col2, _col3 - File Output Operator - compressed: false - GlobalTableId: 0 + outputColumnNames: _col0, _col1, _col10, _col11 + Position of Big Table: 1 + Select Operator + expressions: + expr: _col0 + type: string + expr: _col1 + type: string + expr: _col10 + type: string + expr: _col11 + type: string + outputColumnNames: _col0, _col1, _col2, _col3 + File Output Operator + compressed: false + GlobalTableId: 0 #### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col2,_col3 - columns.types string,string,string,string - escape.delim \ - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false + NumFilesPerFileSink: 1 + table: + input format: org.apache.hadoop.mapred.SequenceFileInputFormat + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat + properties: + columns _col0,_col1,_col2,_col3 + columns.types string,string,string,string + escape.delim \ + TotalFiles: 1 + GatherStats: false + MultiFileSpray: false Local Work: Map Reduce Local Work Needs Tagging: true @@ -290,7 +285,7 @@ name: default.dst_union22_delta name: default.dst_union22_delta Truncated Path -> Alias: - /dst_union22/ds=1 [null-subquery2:subq-subquery2:a] + /dst_union22_delta/ds=1 [null-subquery2:subq-subquery2:b:dst_union22_delta] Stage: Stage-2 Map Reduce @@ -507,205 +502,7 @@ Stats-Aggr Operator #### A masked pattern was here #### - Stage: Stage-1 - Map Reduce - Alias -> Map Operator Tree: - null-subquery2:subq-subquery2:a - TableScan - alias: a - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: - expr: (k1 > 20.0) - type: boolean - Reduce Output Operator - key expressions: - expr: k1 - type: string - sort order: + - Map-reduce partition columns: - expr: k1 - type: string - tag: 0 - value expressions: - expr: k1 - type: string - expr: k2 - type: string - expr: ds - type: string - null-subquery2:subq-subquery2:b:dst_union22_delta - TableScan - alias: dst_union22_delta - GatherStats: false - Filter Operator - isSamplingPred: false - predicate: - expr: ((k0 > 50.0) and (k1 > 20.0)) - type: boolean - Select Operator - expressions: - expr: k1 - type: string - expr: k3 - type: string - expr: k4 - type: string - outputColumnNames: _col1, _col3, _col4 - Reduce Output Operator - key expressions: - expr: _col1 - type: string - sort order: + - Map-reduce partition columns: - expr: _col1 - type: string - tag: 1 - value expressions: - expr: _col3 - type: string - expr: _col4 - type: string - Needs Tagging: true - Path -> Alias: -#### A masked pattern was here #### - Path -> Partition: -#### A masked pattern was here #### - Partition - base file name: ds=1 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 1 - properties: - bucket_count -1 - columns k1,k2,k3,k4 - columns.types string:string:string:string -#### A masked pattern was here #### - name default.dst_union22 - numFiles 1 - numRows 500 - partition_columns ds - rawDataSize 11124 - serialization.ddl struct dst_union22 { string k1, string k2, string k3, string k4} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 11624 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns k1,k2,k3,k4 - columns.types string:string:string:string -#### A masked pattern was here #### - name default.dst_union22 - numFiles 1 - numPartitions 1 - numRows 500 - partition_columns ds - rawDataSize 11124 - serialization.ddl struct dst_union22 { string k1, string k2, string k3, string k4} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 11624 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.dst_union22 - name: default.dst_union22 -#### A masked pattern was here #### - Partition - base file name: ds=1 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - ds 1 - properties: - bucket_count -1 - columns k0,k1,k2,k3,k4,k5 - columns.types string:string:string:string:string:string -#### A masked pattern was here #### - name default.dst_union22_delta - numFiles 1 - numRows 500 - partition_columns ds - rawDataSize 16936 - serialization.ddl struct dst_union22_delta { string k0, string k1, string k2, string k3, string k4, string k5} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 17436 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns k0,k1,k2,k3,k4,k5 - columns.types string:string:string:string:string:string -#### A masked pattern was here #### - name default.dst_union22_delta - numFiles 1 - numPartitions 1 - numRows 500 - partition_columns ds - rawDataSize 16936 - serialization.ddl struct dst_union22_delta { string k0, string k1, string k2, string k3, string k4, string k5} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 17436 -#### A masked pattern was here #### - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.dst_union22_delta - name: default.dst_union22_delta - Reduce Operator Tree: - Join Operator - condition map: - Left Outer Join0 to 1 - condition expressions: - 0 {VALUE._col0} {VALUE._col1} - 1 {VALUE._col3} {VALUE._col4} - filter mappings: - 0 [1, 1] - filter predicates: - 0 {(VALUE._col4 = '1')} - 1 - handleSkewJoin: false - outputColumnNames: _col0, _col1, _col10, _col11 - Select Operator - expressions: - expr: _col0 - type: string - expr: _col1 - type: string - expr: _col10 - type: string - expr: _col11 - type: string - outputColumnNames: _col0, _col1, _col2, _col3 - File Output Operator - compressed: false - GlobalTableId: 0 -#### A masked pattern was here #### - NumFilesPerFileSink: 1 - table: - input format: org.apache.hadoop.mapred.SequenceFileInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - properties: - columns _col0,_col1,_col2,_col3 - columns.types string,string,string,string - escape.delim \ - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - Truncated Path -> Alias: - /dst_union22/ds=1 [null-subquery2:subq-subquery2:a] - /dst_union22_delta/ds=1 [null-subquery2:subq-subquery2:b:dst_union22_delta] - PREHOOK: query: insert overwrite table dst_union22 partition (ds='2') select * from ( @@ -1114,8 +911,6 @@ 239 val_239 239 val_239 2 239 val_239 239 val_239 2 239 val_239 239 val_239 2 -24 val_24 NULL NULL 2 -24 val_24 NULL NULL 2 24 val_24 24 val_24 2 24 val_24 24 val_24 2 241 val_241 241 val_241 2 @@ -1138,8 +933,6 @@ 256 val_256 256 val_256 2 257 val_257 257 val_257 2 258 val_258 258 val_258 2 -26 val_26 NULL NULL 2 -26 val_26 NULL NULL 2 26 val_26 26 val_26 2 26 val_26 26 val_26 2 260 val_260 260 val_260 2 @@ -1150,7 +943,6 @@ 265 val_265 265 val_265 2 265 val_265 265 val_265 2 266 val_266 266 val_266 2 -27 val_27 NULL NULL 2 27 val_27 27 val_27 2 272 val_272 272 val_272 2 272 val_272 272 val_272 2 @@ -1187,7 +979,6 @@ 278 val_278 278 val_278 2 278 val_278 278 val_278 2 278 val_278 278 val_278 2 -28 val_28 NULL NULL 2 28 val_28 28 val_28 2 280 val_280 280 val_280 2 280 val_280 280 val_280 2 @@ -1223,7 +1014,6 @@ 298 val_298 298 val_298 2 298 val_298 298 val_298 2 298 val_298 298 val_298 2 -30 val_30 NULL NULL 2 30 val_30 30 val_30 2 302 val_302 302 val_302 2 305 val_305 305 val_305 2 @@ -1292,7 +1082,6 @@ 327 val_327 327 val_327 2 327 val_327 327 val_327 2 327 val_327 327 val_327 2 -33 val_33 NULL NULL 2 33 val_33 33 val_33 2 331 val_331 331 val_331 2 331 val_331 331 val_331 2 @@ -1307,7 +1096,6 @@ 336 val_336 336 val_336 2 338 val_338 338 val_338 2 339 val_339 339 val_339 2 -34 val_34 NULL NULL 2 34 val_34 34 val_34 2 341 val_341 341 val_341 2 342 val_342 342 val_342 2 @@ -1344,9 +1132,6 @@ 348 val_348 348 val_348 2 348 val_348 348 val_348 2 348 val_348 348 val_348 2 -35 val_35 NULL NULL 2 -35 val_35 NULL NULL 2 -35 val_35 NULL NULL 2 35 val_35 35 val_35 2 35 val_35 35 val_35 2 35 val_35 35 val_35 2 @@ -1375,8 +1160,6 @@ 369 val_369 369 val_369 2 369 val_369 369 val_369 2 369 val_369 369 val_369 2 -37 val_37 NULL NULL 2 -37 val_37 NULL NULL 2 37 val_37 37 val_37 2 37 val_37 37 val_37 2 373 val_373 373 val_373 2 @@ -1491,7 +1274,6 @@ 409 val_409 409 val_409 2 409 val_409 409 val_409 2 409 val_409 409 val_409 2 -41 val_41 NULL NULL 2 41 val_41 41 val_41 2 411 val_411 411 val_411 2 413 val_413 413 val_413 2 @@ -1513,8 +1295,6 @@ 417 val_417 417 val_417 2 418 val_418 418 val_418 2 419 val_419 419 val_419 2 -42 val_42 NULL NULL 2 -42 val_42 NULL NULL 2 42 val_42 42 val_42 2 42 val_42 42 val_42 2 421 val_421 421 val_421 2 @@ -1527,7 +1307,6 @@ 429 val_429 429 val_429 2 429 val_429 429 val_429 2 429 val_429 429 val_429 2 -43 val_43 NULL NULL 2 43 val_43 43 val_43 2 430 val_430 430 val_430 2 430 val_430 430 val_430 2 @@ -1564,7 +1343,6 @@ 439 val_439 439 val_439 2 439 val_439 439 val_439 2 439 val_439 439 val_439 2 -44 val_44 NULL NULL 2 44 val_44 44 val_44 2 443 val_443 443 val_443 2 444 val_444 444 val_444 2 @@ -1652,7 +1430,6 @@ 469 val_469 469 val_469 2 469 val_469 469 val_469 2 469 val_469 469 val_469 2 -47 val_47 NULL NULL 2 47 val_47 47 val_47 2 470 val_470 470 val_470 2 472 val_472 472 val_472 2 Index: ql/src/test/results/clientpositive/join31.q.out =================================================================== --- ql/src/test/results/clientpositive/join31.q.out (revision 1445446) +++ ql/src/test/results/clientpositive/join31.q.out (working copy) @@ -26,15 +26,11 @@ STAGE DEPENDENCIES: Stage-1 is a root stage - Stage-8 depends on stages: Stage-1, Stage-5 , consists of Stage-9, Stage-10, Stage-2 - Stage-9 has a backup stage: Stage-2 - Stage-6 depends on stages: Stage-9 - Stage-3 depends on stages: Stage-2, Stage-6, Stage-7 + Stage-7 depends on stages: Stage-1, Stage-5 + Stage-6 depends on stages: Stage-7 + Stage-3 depends on stages: Stage-6 Stage-0 depends on stages: Stage-3 Stage-4 depends on stages: Stage-0 - Stage-10 has a backup stage: Stage-2 - Stage-7 depends on stages: Stage-10 - Stage-2 Stage-5 is a root stage STAGE PLANS: @@ -92,17 +88,14 @@ input format: org.apache.hadoop.mapred.SequenceFileInputFormat output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat - Stage: Stage-8 - Conditional Operator - - Stage: Stage-9 + Stage: Stage-7 Map Reduce Local Work Alias -> Map Local Tables: - $INTNAME + $INTNAME1 Fetch Operator limit: -1 Alias -> Map Local Operator Tree: - $INTNAME + $INTNAME1 HashTable Sink Operator condition expressions: 0 {_col0} @@ -111,12 +104,12 @@ keys: 0 [Column[_col0]] 1 [Column[_col0]] - Position of Big Table: 0 + Position of Big Table: 1 Stage: Stage-6 Map Reduce Alias -> Map Operator Tree: - $INTNAME1 + $INTNAME Map Join Operator condition map: Inner Join 0 to 1 @@ -128,7 +121,7 @@ 0 [Column[_col0]] 1 [Column[_col0]] outputColumnNames: _col0 - Position of Big Table: 0 + Position of Big Table: 1 Select Operator expressions: expr: _col0 @@ -214,119 +207,6 @@ Stage: Stage-4 Stats-Aggr Operator - Stage: Stage-10 - Map Reduce Local Work - Alias -> Map Local Tables: - $INTNAME1 - Fetch Operator - limit: -1 - Alias -> Map Local Operator Tree: - $INTNAME1 - HashTable Sink Operator - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - Position of Big Table: 1 - - Stage: Stage-7 - Map Reduce - Alias -> Map Operator Tree: - $INTNAME - Map Join Operator - condition map: - Inner Join 0 to 1 - condition expressions: - 0 {_col0} - 1 - handleSkewJoin: false - keys: - 0 [Column[_col0]] - 1 [Column[_col0]] - outputColumnNames: _col0 - Position of Big Table: 1 - 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 - Local Work: - Map Reduce Local Work - - Stage: Stage-2 - Map Reduce - Alias -> Map Operator Tree: - $INTNAME - Reduce Output Operator - key expressions: - expr: _col0 - type: string - sort order: + - Map-reduce partition columns: - expr: _col0 - type: string - tag: 1 - $INTNAME1 - Reduce Output Operator - key expressions: - expr: _col0 - type: string - sort order: + - Map-reduce partition columns: - expr: _col0 - type: string - tag: 0 - value expressions: - expr: _col0 - type: string - 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 - 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-5 Map Reduce Alias -> Map Operator Tree: Index: ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java =================================================================== --- ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java (revision 1445446) +++ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java (working copy) @@ -18,6 +18,7 @@ package org.apache.hadoop.hive.ql.optimizer.physical; import java.io.ByteArrayInputStream; +import java.io.IOException; import java.io.InputStream; import java.io.Serializable; import java.io.UnsupportedEncodingException; @@ -50,8 +51,7 @@ import org.apache.hadoop.hive.ql.parse.QBJoinTree; import org.apache.hadoop.hive.ql.parse.SemanticException; import org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin; -import - org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin.ConditionalResolverCommonJoinCtx; +import org.apache.hadoop.hive.ql.plan.ConditionalResolverCommonJoin.ConditionalResolverCommonJoinCtx; import org.apache.hadoop.hive.ql.plan.ConditionalWork; import org.apache.hadoop.hive.ql.plan.JoinDesc; import org.apache.hadoop.hive.ql.plan.MapredLocalWork; @@ -98,6 +98,7 @@ * task would be merged with the map-reduce task to create a single map-reduce task. */ public class CommonJoinResolver implements PhysicalPlanResolver { + @Override public PhysicalContext resolve(PhysicalContext pctx) throws SemanticException { @@ -121,6 +122,8 @@ */ class CommonJoinTaskDispatcher implements Dispatcher { + long largestTableSize = 0; + HashMap aliasToSize = null; private final PhysicalContext physicalContext; public CommonJoinTaskDispatcher(PhysicalContext context) { @@ -146,7 +149,8 @@ * A task and its child task has been converted from join to mapjoin. * See if the two tasks can be merged. */ - private void mergeMapJoinTaskWithChildMapJoinTask(MapRedTask task) { + private void mergeMapJoinTaskWithChildMapJoinTask(MapRedTask task, Configuration conf) + throws IOException { MapRedTask childTask = (MapRedTask)task.getChildTasks().get(0); MapredWork work = task.getWork(); MapredLocalWork localWork = work.getMapLocalWork(); @@ -175,10 +179,7 @@ FileSinkOperator fop = (FileSinkOperator)op; String workDir = fop.getConf().getDirName(); - Map> childPathToAliases = childWork.getPathToAliases(); - if (childPathToAliases.size() > 1) { - return; - } + HashMap> childPathToAliases = childWork.getPathToAliases(); // The filesink writes to a different directory if (!childPathToAliases.keySet().iterator().next().equals(workDir)) { @@ -191,11 +192,22 @@ return; } - // Merge the trees - if (childWork.getAliasToWork().size() > 1) { - return; + long mapJoinSize = HiveConf.getLongVar(conf, + HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); + + long localTableTotalSize = 0; + for (String alias : localWork.getAliasToWork().keySet()) { + localTableTotalSize += aliasToSize.get(alias); } + for (String alias : childLocalWork.getAliasToWork().keySet()) { + long size = aliasToSize.get(alias); + localTableTotalSize += size; + if (localTableTotalSize > mapJoinSize) { + return; + } + } + Operator childAliasOp = childWork.getAliasToWork().values().iterator().next(); if (fop.getParentOperators().size() > 1) { @@ -234,6 +246,56 @@ } } + private int getBigTablePosition(Map> aliasToWork, + HashMap aliasToSize, MapredWork currWork, JoinOperator joinOp, + Configuration conf) { + // If sizes of atleast n-1 tables in a n-way join is known, and their sum is smaller than + // the threshold size, convert the join into map-join and don't create a conditional task + boolean convertJoinMapJoin = HiveConf.getBoolVar(conf, + HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASK); + int bigTablePosition = -1; + if (convertJoinMapJoin) { + // This is the threshold that the user has specified to fit in mapjoin + long mapJoinSize = HiveConf.getLongVar(conf, + HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); + + boolean bigTableFound = false; + largestTableSize = 0; + long sumTableSizes = 0; + for (String alias : aliasToWork.keySet()) { + Long size = aliasToSize.get(alias); + // The size is not available at compile time if the input is a sub-query. + // If the size of atleast n-1 inputs for a n-way join are available at compile time, + // and the sum of them is less than the specified threshold, then convert the join + // into a map-join without the conditional task. + if ((size == null) || (size > mapJoinSize)) { + sumTableSizes += largestTableSize; + if (bigTableFound || (sumTableSizes > mapJoinSize)) { + convertJoinMapJoin = false; + break; + } + bigTableFound = true; + bigTablePosition = getPosition(currWork, joinOp, alias); + largestTableSize = mapJoinSize + 1; + } else { + if (size > largestTableSize) { + sumTableSizes += largestTableSize; + largestTableSize = size; + bigTablePosition = getPosition(currWork, joinOp, alias); + } else { + sumTableSizes += size; + } + if (sumTableSizes > mapJoinSize) { + convertJoinMapJoin = false; + break; + } + } + } + } + + return bigTablePosition; + } + // create map join task and set big table as bigTablePosition private ObjectPair convertTaskToMapJoinTask(String xml, int bigTablePosition) throws UnsupportedEncodingException, SemanticException { @@ -283,7 +345,11 @@ int numAliases = order.length; long aliasTotalKnownInputSize = 0; - HashMap aliasToSize = new HashMap(); + + if (aliasToSize == null) { + aliasToSize = new HashMap(); + } + try { // go over all the input paths, and calculate a known total size, known // size for each input alias. @@ -318,49 +384,12 @@ } Configuration conf = context.getConf(); + int bigTablePosition = getBigTablePosition(aliasToWork, aliasToSize, + currWork, joinOp, conf); - // If sizes of atleast n-1 tables in a n-way join is known, and their sum is smaller than - // the threshold size, convert the join into map-join and don't create a conditional task - boolean convertJoinMapJoin = HiveConf.getBoolVar(conf, - HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASK); - int bigTablePosition = -1; - if (convertJoinMapJoin) { - // This is the threshold that the user has specified to fit in mapjoin - long mapJoinSize = HiveConf.getLongVar(conf, - HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD); - - boolean bigTableFound = false; - long largestTableSize = 0; - long sumTableSizes = 0; - for (String alias : aliasToWork.keySet()) { - Long size = aliasToSize.get(alias); - // The size is not available at compile time if the input is a sub-query. - // If the size of atleast n-1 inputs for a n-way join are available at compile time, - // and the sum of them is less than the specified threshold, then convert the join - // into a map-join without the conditional task. - if ((size == null) || (size > mapJoinSize)) { - sumTableSizes += largestTableSize; - if (bigTableFound || (sumTableSizes > mapJoinSize)) { - convertJoinMapJoin = false; - break; - } - bigTableFound = true; - bigTablePosition = getPosition(currWork, joinOp, alias); - largestTableSize = mapJoinSize + 1; - } else { - if (size > largestTableSize) { - sumTableSizes += largestTableSize; - largestTableSize = size; - bigTablePosition = getPosition(currWork, joinOp, alias); - } else { - sumTableSizes += size; - } - if (sumTableSizes > mapJoinSize) { - convertJoinMapJoin = false; - break; - } - } - } + boolean convertJoinMapJoin = true; + if (bigTablePosition == -1) { + convertJoinMapJoin = false; } String bigTableAlias = null; @@ -381,7 +410,7 @@ // followed by a mapjoin can be performed in a single MR job. if ((newTask.getChildTasks() != null) && (newTask.getChildTasks().size() == 1) && (newTask.getChildTasks().get(0).getTaskTag() == Task.MAPJOIN_ONLY_NOBACKUP)) { - mergeMapJoinTaskWithChildMapJoinTask(newTask); + mergeMapJoinTaskWithChildMapJoinTask(newTask, conf); } return newTask;