You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2013/03/20 17:02:09 UTC

svn commit: r1458903 [1/6] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/

Author: namit
Date: Wed Mar 20 16:02:08 2013
New Revision: 1458903

URL: http://svn.apache.org/r1458903
Log:
HIVE-4146 bug with hive.auto.convert.join.noconditionaltask with outer joins
(Namit via Gang Tim Liu)


Added:
    hive/trunk/ql/src/test/queries/clientpositive/join41.q
    hive/trunk/ql/src/test/results/clientpositive/join41.q.out
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
    hive/trunk/ql/src/test/queries/clientpositive/auto_join25.q
    hive/trunk/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q
    hive/trunk/ql/src/test/queries/clientpositive/mapjoin_hook.q
    hive/trunk/ql/src/test/results/clientpositive/auto_join0.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join1.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join10.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join11.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join12.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join13.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join14_hadoop20.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join15.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join16.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join17.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join19.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join2.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join20.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join21.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join22.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join23.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join24.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join25.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join26.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join27.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join28.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join29.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join3.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join4.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join5.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join8.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_join9.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_10.q.out
    hive/trunk/ql/src/test/results/clientpositive/auto_sortmerge_join_9.q.out
    hive/trunk/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
    hive/trunk/ql/src/test/results/clientpositive/join_reorder4.q.out
    hive/trunk/ql/src/test/results/clientpositive/mapjoin_filter_on_outerjoin.q.out
    hive/trunk/ql/src/test/results/clientpositive/mapjoin_hook.q.out
    hive/trunk/ql/src/test/results/clientpositive/multiMapJoin1.q.out
    hive/trunk/ql/src/test/results/clientpositive/multi_join_union.q.out
    hive/trunk/ql/src/test/results/clientpositive/reduce_deduplicate_exclude_join.q.out

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Wed Mar 20 16:02:08 2013
@@ -484,7 +484,7 @@ public class HiveConf extends Configurat
 
     HIVESKEWJOIN("hive.optimize.skewjoin", false),
     HIVECONVERTJOIN("hive.auto.convert.join", true),
-    HIVECONVERTJOINNOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", false),
+    HIVECONVERTJOINNOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", true),
     HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD("hive.auto.convert.join.noconditionaltask.size",
         10000000L),
     HIVESKEWJOINKEY("hive.skewjoin.key", 100000),

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Wed Mar 20 16:02:08 2013
@@ -810,7 +810,7 @@
 
 <property>
   <name>hive.auto.convert.join.noconditionaltask</name>
-  <value>false</value>
+  <value>true</value>
   <description>Whether Hive enable the optimization about converting common join into mapjoin based on the input file 
     size. If this paramater is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the
     specified size, the join is directly converted to a mapjoin (there is no conditional task).

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/CommonJoinResolver.java Wed Mar 20 16:02:08 2013
@@ -329,31 +329,35 @@ public class CommonJoinResolver implemen
               HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD);
 
           boolean bigTableFound = false;
-          long largestTableSize = 0;
+          long largestBigTableCandidateSize = 0;
           long sumTableSizes = 0;
           for (String alias : aliasToWork.keySet()) {
+            int tablePosition = getPosition(currWork, joinOp, alias);
+            boolean bigTableCandidate = bigTableCandidates.contains(tablePosition);
             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)) {
+              sumTableSizes += largestBigTableCandidateSize;
+              if (bigTableFound || (sumTableSizes > mapJoinSize) || !bigTableCandidate) {
                 convertJoinMapJoin = false;
                 break;
               }
               bigTableFound = true;
-              bigTablePosition = getPosition(currWork, joinOp, alias);
-              largestTableSize = mapJoinSize + 1;
+              bigTablePosition = tablePosition;
+              largestBigTableCandidateSize = mapJoinSize + 1;
             } else {
-              if (size > largestTableSize) {
-                sumTableSizes += largestTableSize;
-                largestTableSize = size;
-                bigTablePosition = getPosition(currWork, joinOp, alias);
-              } else {
+              if (bigTableCandidate && size > largestBigTableCandidateSize) {
+                bigTablePosition = tablePosition;
+                sumTableSizes += largestBigTableCandidateSize;
+                largestBigTableCandidateSize = size;
+              }
+              else {
                 sumTableSizes += size;
               }
+
               if (sumTableSizes > mapJoinSize) {
                 convertJoinMapJoin = false;
                 break;

Modified: hive/trunk/ql/src/test/queries/clientpositive/auto_join25.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/auto_join25.q?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/auto_join25.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/auto_join25.q Wed Mar 20 16:02:08 2013
@@ -1,7 +1,9 @@
 set hive.auto.convert.join = true;
 set hive.mapjoin.localtask.max.memory.usage = 0.0001;
 set hive.mapjoin.check.memory.rows = 2;
+set hive.auto.convert.join.noconditionaltask = false;
 
+-- This test tests the scenario when the mapper dies. So, create a conditional task for the mapjoin
 CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
 
 FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key)

Modified: hive/trunk/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q Wed Mar 20 16:02:08 2013
@@ -16,8 +16,10 @@ DESCRIBE FORMATTED test_table PARTITION 
 
 set hive.mapjoin.check.memory.rows=1;
 set hive.mapjoin.localtask.max.memory.usage = 0.0001;
+set hive.auto.convert.join.noconditionaltask = false;
 
--- Tests a join which is not converted to a map join, the output should be bucketed and sorted
+-- This test tests the scenario when the mapper dies. So, create a conditional task for the mapjoin.
+-- Tests a join which is not converted to a map join, the output should be bucketed and sorted.
 
 INSERT OVERWRITE TABLE test_table PARTITION (part = '1') 
 SELECT a.key, b.value FROM src a JOIN src b ON a.key = b.key;

Added: hive/trunk/ql/src/test/queries/clientpositive/join41.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/join41.q?rev=1458903&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/join41.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/join41.q Wed Mar 20 16:02:08 2013
@@ -0,0 +1,16 @@
+create table s1 as select * from src where key = 0;
+
+set hive.auto.convert.join.noconditionaltask=false;
+EXPLAIN
+SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10);
+SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10);
+
+set hive.auto.convert.join.noconditionaltask=true;
+
+-- Make sure the big table is chosen correctly as part of HIVE-4146
+EXPLAIN
+SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10);
+SELECT * FROM s1 src1 LEFT OUTER JOIN s1 src2 ON (src1.key = src2.key AND src2.key > 10);
+
+
+

Modified: hive/trunk/ql/src/test/queries/clientpositive/mapjoin_hook.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/mapjoin_hook.q?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/mapjoin_hook.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/mapjoin_hook.q Wed Mar 20 16:02:08 2013
@@ -15,6 +15,7 @@ INSERT OVERWRITE TABLE dest1 SELECT src1
 
 set hive.mapjoin.localtask.max.memory.usage = 0.0001;
 set hive.mapjoin.check.memory.rows = 2;
+set hive.auto.convert.join.noconditionaltask = false;
 
 
 FROM srcpart src1 JOIN src src2 ON (src1.key = src2.key)

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join0.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join0.q.out?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join0.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join0.q.out Wed Mar 20 16:02:08 2013
@@ -24,21 +24,14 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL src) key) 10)))) src1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR TOK_ALLCOLREF)) (TOK_WHERE (< (. (TOK_TABLE_OR_COL src) key) 10)))) src2))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key) k1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) value) v1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) key) k2) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) value) v2)) (TOK_SORTBY (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL k1)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL v1)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL k2)) (TOK_TABSORTCOLNAMEASC (TOK_TABLE_OR_COL v2))))) a)) (TOK_INSERT 
 (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL a) k1) (. (TOK_TABLE_OR_COL a) v1) (. (TOK_TABLE_OR_COL a) k2) (. (TOK_TABLE_OR_COL a) v2)))))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage , consists of Stage-8, Stage-9, Stage-1
-  Stage-8 has a backup stage: Stage-1
-  Stage-5 depends on stages: Stage-8
-  Stage-2 depends on stages: Stage-1, Stage-5, Stage-6
+  Stage-6 is a root stage
+  Stage-5 depends on stages: Stage-6
+  Stage-2 depends on stages: Stage-5
   Stage-3 depends on stages: Stage-2
-  Stage-9 has a backup stage: Stage-1
-  Stage-6 depends on stages: Stage-9
-  Stage-1
   Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-7
-    Conditional Operator
-
-  Stage: Stage-8
+  Stage: Stage-6
     Map Reduce Local Work
       Alias -> Map Local Tables:
         a:src2:src 
@@ -198,160 +191,6 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 
-  Stage: Stage-9
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        a:src1:src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        a:src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 10.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0} {_col1}
-                    1 {_col0} {_col1}
-                  handleSkewJoin: false
-                  keys:
-                    0 []
-                    1 []
-                  Position of Big Table: 1
-
-  Stage: Stage-6
-    Map Reduce
-      Alias -> Map Operator Tree:
-        a:src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 10.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  condition expressions:
-                    0 {_col0} {_col1}
-                    1 {_col0} {_col1}
-                  handleSkewJoin: false
-                  keys:
-                    0 []
-                    1 []
-                  outputColumnNames: _col0, _col1, _col2, _col3
-                  Position of Big Table: 1
-                  Select Operator
-                    expressions:
-                          expr: _col0
-                          type: string
-                          expr: _col1
-                          type: string
-                          expr: _col2
-                          type: string
-                          expr: _col3
-                          type: string
-                    outputColumnNames: _col0, _col1, _col2, _col3
-                    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-1
-    Map Reduce
-      Alias -> Map Operator Tree:
-        a:src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 10.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                Reduce Output Operator
-                  sort order: 
-                  tag: 0
-                  value expressions:
-                        expr: _col0
-                        type: string
-                        expr: _col1
-                        type: string
-        a:src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 10.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                Reduce Output Operator
-                  sort order: 
-                  tag: 1
-                  value expressions:
-                        expr: _col0
-                        type: string
-                        expr: _col1
-                        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, _col2, _col3
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: string
-                  expr: _col1
-                  type: string
-                  expr: _col2
-                  type: string
-                  expr: _col3
-                  type: string
-            outputColumnNames: _col0, _col1, _col2, _col3
-            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-0
     Fetch Operator
       limit: -1

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join1.q.out?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join1.q.out Wed Mar 20 16:02:08 2013
@@ -15,29 +15,22 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src) src1) (TOK_TABREF (TOK_TABNAME src) src2) (= (. (TOK_TABLE_OR_COL src1) key) (. (TOK_TABLE_OR_COL src2) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest_j1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src1) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src2) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1
-  Stage-7 has a backup stage: Stage-1
-  Stage-4 depends on stages: Stage-7
-  Stage-0 depends on stages: Stage-1, Stage-4, Stage-5
+  Stage-5 is a root stage
+  Stage-4 depends on stages: Stage-5
+  Stage-0 depends on stages: Stage-4
   Stage-2 depends on stages: Stage-0
-  Stage-8 has a backup stage: Stage-1
-  Stage-5 depends on stages: Stage-8
-  Stage-1
 
 STAGE PLANS:
-  Stage: Stage-6
-    Conditional Operator
-
-  Stage: Stage-7
+  Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2 
+        src1 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2 
+        src1 
           TableScan
-            alias: src2
+            alias: src1
             HashTable Sink Operator
               condition expressions:
                 0 {key}
@@ -46,14 +39,14 @@ STAGE PLANS:
               keys:
                 0 [Column[key]]
                 1 [Column[key]]
-              Position of Big Table: 0
+              Position of Big Table: 1
 
   Stage: Stage-4
     Map Reduce
       Alias -> Map Operator Tree:
-        src1 
+        src2 
           TableScan
-            alias: src1
+            alias: src2
             Map Join Operator
               condition map:
                    Inner Join 0 to 1
@@ -65,7 +58,7 @@ STAGE PLANS:
                 0 [Column[key]]
                 1 [Column[key]]
               outputColumnNames: _col0, _col5
-              Position of Big Table: 0
+              Position of Big Table: 1
               Select Operator
                 expressions:
                       expr: UDFToInteger(_col0)
@@ -97,119 +90,6 @@ STAGE PLANS:
   Stage: Stage-2
     Stats-Aggr Operator
 
-  Stage: Stage-8
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        src1 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        src1 
-          TableScan
-            alias: src1
-            HashTable Sink Operator
-              condition expressions:
-                0 {key}
-                1 {value}
-              handleSkewJoin: false
-              keys:
-                0 [Column[key]]
-                1 [Column[key]]
-              Position of Big Table: 1
-
-  Stage: Stage-5
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src2 
-          TableScan
-            alias: src2
-            Map Join Operator
-              condition map:
-                   Inner Join 0 to 1
-              condition expressions:
-                0 {key}
-                1 {value}
-              handleSkewJoin: false
-              keys:
-                0 [Column[key]]
-                1 [Column[key]]
-              outputColumnNames: _col0, _col5
-              Position of Big Table: 1
-              Select Operator
-                expressions:
-                      expr: UDFToInteger(_col0)
-                      type: int
-                      expr: _col5
-                      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.dest_j1
-      Local Work:
-        Map Reduce Local Work
-
-  Stage: Stage-1
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src1 
-          TableScan
-            alias: src1
-            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
-        src2 
-          TableScan
-            alias: src2
-            Reduce Output Operator
-              key expressions:
-                    expr: key
-                    type: string
-              sort order: +
-              Map-reduce partition columns:
-                    expr: key
-                    type: string
-              tag: 1
-              value expressions:
-                    expr: value
-                    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: UDFToInteger(_col0)
-                  type: int
-                  expr: _col5
-                  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.dest_j1
 
 PREHOOK: query: FROM src src1 JOIN src src2 ON (src1.key = src2.key)
 INSERT OVERWRITE TABLE dest_j1 SELECT src1.key, src2.value

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join10.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join10.q.out?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join10.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join10.q.out Wed Mar 20 16:02:08 2013
@@ -18,20 +18,13 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME src)))))) x) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME src)))))) Y) (= (. (TOK_TABLE_OR_COL x) key) (. (TOK_TABLE_OR_COL Y) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL Y) key) (. (TOK_TABLE_OR_COL Y) value)))))))
 
 STAGE DEPENDENCIES:
-  Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1
-  Stage-7 has a backup stage: Stage-1
-  Stage-4 depends on stages: Stage-7
-  Stage-2 depends on stages: Stage-1, Stage-4, Stage-5
-  Stage-8 has a backup stage: Stage-1
-  Stage-5 depends on stages: Stage-8
-  Stage-1
+  Stage-5 is a root stage
+  Stage-4 depends on stages: Stage-5
+  Stage-2 depends on stages: Stage-4
   Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-6
-    Conditional Operator
-
-  Stage: Stage-7
+  Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
         y:src 
@@ -132,151 +125,6 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 
-  Stage: Stage-8
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        x:src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        x:src 
-          TableScan
-            alias: src
-            Select Operator
-              expressions:
-                    expr: key
-                    type: string
-              outputColumnNames: _col0
-              HashTable Sink Operator
-                condition expressions:
-                  0 
-                  1 {_col0} {_col1}
-                handleSkewJoin: false
-                keys:
-                  0 [Column[_col0]]
-                  1 [Column[_col0]]
-                Position of Big Table: 1
-
-  Stage: Stage-5
-    Map Reduce
-      Alias -> Map Operator Tree:
-        y:src 
-          TableScan
-            alias: src
-            Select Operator
-              expressions:
-                    expr: key
-                    type: string
-                    expr: value
-                    type: string
-              outputColumnNames: _col0, _col1
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                condition expressions:
-                  0 
-                  1 {_col0} {_col1}
-                handleSkewJoin: false
-                keys:
-                  0 [Column[_col0]]
-                  1 [Column[_col0]]
-                outputColumnNames: _col2, _col3
-                Position of Big Table: 1
-                Select Operator
-                  expressions:
-                        expr: _col2
-                        type: string
-                        expr: _col3
-                        type: string
-                  outputColumnNames: _col2, _col3
-                  Group By Operator
-                    aggregations:
-                          expr: sum(hash(_col2,_col3))
-                    bucketGroup: false
-                    mode: hash
-                    outputColumnNames: _col0
-                    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-1
-    Map Reduce
-      Alias -> Map Operator Tree:
-        x:src 
-          TableScan
-            alias: src
-            Select Operator
-              expressions:
-                    expr: key
-                    type: string
-              outputColumnNames: _col0
-              Reduce Output Operator
-                key expressions:
-                      expr: _col0
-                      type: string
-                sort order: +
-                Map-reduce partition columns:
-                      expr: _col0
-                      type: string
-                tag: 0
-        y:src 
-          TableScan
-            alias: src
-            Select Operator
-              expressions:
-                    expr: key
-                    type: string
-                    expr: value
-                    type: string
-              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: _col0
-                      type: string
-                      expr: _col1
-                      type: string
-      Reduce Operator Tree:
-        Join Operator
-          condition map:
-               Inner Join 0 to 1
-          condition expressions:
-            0 
-            1 {VALUE._col0} {VALUE._col1}
-          handleSkewJoin: false
-          outputColumnNames: _col2, _col3
-          Select Operator
-            expressions:
-                  expr: _col2
-                  type: string
-                  expr: _col3
-                  type: string
-            outputColumnNames: _col2, _col3
-            Group By Operator
-              aggregations:
-                    expr: sum(hash(_col2,_col3))
-              bucketGroup: false
-              mode: hash
-              outputColumnNames: _col0
-              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-0
     Fetch Operator
       limit: -1

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join11.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join11.q.out?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join11.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join11.q.out Wed Mar 20 16:02:08 2013
@@ -18,27 +18,20 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c2)))) src1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c4)))) src2) (AND (= (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src2) c3)) (< (. (TOK_TABLE_OR_COL src1) c1) 100)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src2) c4)))))))
 
 STAGE DEPENDENCIES:
-  Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1
-  Stage-7 has a backup stage: Stage-1
-  Stage-4 depends on stages: Stage-7
-  Stage-2 depends on stages: Stage-1, Stage-4, Stage-5
-  Stage-8 has a backup stage: Stage-1
-  Stage-5 depends on stages: Stage-8
-  Stage-1
+  Stage-5 is a root stage
+  Stage-4 depends on stages: Stage-5
+  Stage-2 depends on stages: Stage-4
   Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-6
-    Conditional Operator
-
-  Stage: Stage-7
+  Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2:src 
+        src1:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2:src 
+        src1:src 
           TableScan
             alias: src
             Filter Operator
@@ -49,9 +42,7 @@ STAGE PLANS:
                 expressions:
                       expr: key
                       type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col0
                 HashTable Sink Operator
                   condition expressions:
                     0 {_col0}
@@ -60,12 +51,12 @@ STAGE PLANS:
                   keys:
                     0 [Column[_col0]]
                     1 [Column[_col0]]
-                  Position of Big Table: 0
+                  Position of Big Table: 1
 
   Stage: Stage-4
     Map Reduce
       Alias -> Map Operator Tree:
-        src1:src 
+        src2:src 
           TableScan
             alias: src
             Filter Operator
@@ -76,7 +67,9 @@ STAGE PLANS:
                 expressions:
                       expr: key
                       type: string
-                outputColumnNames: _col0
+                      expr: value
+                      type: string
+                outputColumnNames: _col0, _col1
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -88,7 +81,7 @@ STAGE PLANS:
                     0 [Column[_col0]]
                     1 [Column[_col0]]
                   outputColumnNames: _col0, _col3
-                  Position of Big Table: 0
+                  Position of Big Table: 1
                   Select Operator
                     expressions:
                           expr: _col0
@@ -140,168 +133,6 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 
-  Stage: Stage-8
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        src1:src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                  Position of Big Table: 1
-
-  Stage: Stage-5
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                  outputColumnNames: _col0, _col3
-                  Position of Big Table: 1
-                  Select Operator
-                    expressions:
-                          expr: _col0
-                          type: string
-                          expr: _col3
-                          type: string
-                    outputColumnNames: _col0, _col3
-                    Group By Operator
-                      aggregations:
-                            expr: sum(hash(_col0,_col3))
-                      bucketGroup: false
-                      mode: hash
-                      outputColumnNames: _col0
-                      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-1
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                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
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                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: 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, _col3
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: string
-                  expr: _col3
-                  type: string
-            outputColumnNames: _col0, _col3
-            Group By Operator
-              aggregations:
-                    expr: sum(hash(_col0,_col3))
-              bucketGroup: false
-              mode: hash
-              outputColumnNames: _col0
-              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-0
     Fetch Operator
       limit: -1

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join12.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join12.q.out?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join12.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join12.q.out Wed Mar 20 16:02:08 2013
@@ -24,45 +24,34 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c2)))) src1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c4)))) src2) (AND (= (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src2) c3)) (< (. (TOK_TABLE_OR_COL src1) c1) 100))) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c5) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c6)))) src3) (AND (= (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src3) c5)) (< (. (TOK_TABLE_OR_COL src3) c5) 80)))) (TOK_
 INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src2) c4)))))))
 
 STAGE DEPENDENCIES:
-  Stage-8 is a root stage , consists of Stage-9, Stage-10, Stage-11, Stage-1
-  Stage-9 has a backup stage: Stage-1
-  Stage-5 depends on stages: Stage-9
-  Stage-2 depends on stages: Stage-1, Stage-5, Stage-6, Stage-7
-  Stage-10 has a backup stage: Stage-1
-  Stage-6 depends on stages: Stage-10
-  Stage-11 has a backup stage: Stage-1
-  Stage-7 depends on stages: Stage-11
-  Stage-1
+  Stage-6 is a root stage
+  Stage-5 depends on stages: Stage-6
+  Stage-2 depends on stages: Stage-5
   Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-8
-    Conditional Operator
-
-  Stage: Stage-9
+  Stage: Stage-6
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2:src 
+        src1:src 
           Fetch Operator
             limit: -1
         src3:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2:src 
+        src1:src 
           TableScan
             alias: src
             Filter Operator
               predicate:
-                  expr: ((key < 100.0) and (key < 80.0))
+                  expr: ((key < 80.0) and (key < 100.0))
                   type: boolean
               Select Operator
                 expressions:
                       expr: key
                       type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
+                outputColumnNames: _col0
                 HashTable Sink Operator
                   condition expressions:
                     0 {_col0}
@@ -73,7 +62,7 @@ STAGE PLANS:
                     0 [Column[_col0]]
                     1 [Column[_col0]]
                     2 [Column[_col0]]
-                  Position of Big Table: 0
+                  Position of Big Table: 1
         src3:src 
           TableScan
             alias: src
@@ -96,23 +85,25 @@ STAGE PLANS:
                     0 [Column[_col0]]
                     1 [Column[_col0]]
                     2 [Column[_col0]]
-                  Position of Big Table: 0
+                  Position of Big Table: 1
 
   Stage: Stage-5
     Map Reduce
       Alias -> Map Operator Tree:
-        src1:src 
+        src2:src 
           TableScan
             alias: src
             Filter Operator
               predicate:
-                  expr: ((key < 80.0) and (key < 100.0))
+                  expr: ((key < 100.0) and (key < 80.0))
                   type: boolean
               Select Operator
                 expressions:
                       expr: key
                       type: string
-                outputColumnNames: _col0
+                      expr: value
+                      type: string
+                outputColumnNames: _col0, _col1
                 Map Join Operator
                   condition map:
                        Inner Join 0 to 1
@@ -127,7 +118,7 @@ STAGE PLANS:
                     1 [Column[_col0]]
                     2 [Column[_col0]]
                   outputColumnNames: _col0, _col3
-                  Position of Big Table: 0
+                  Position of Big Table: 1
                   Select Operator
                     expressions:
                           expr: _col0
@@ -179,333 +170,6 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 
-  Stage: Stage-10
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        src1:src 
-          Fetch Operator
-            limit: -1
-        src3:src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 80.0) and (key < 100.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                    2 
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                    2 [Column[_col0]]
-                  Position of Big Table: 1
-        src3:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 100.0) and (key < 80.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                    2 
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                    2 [Column[_col0]]
-                  Position of Big Table: 1
-
-  Stage: Stage-6
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 100.0) and (key < 80.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                       Inner Join 0 to 2
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                    2 
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                    2 [Column[_col0]]
-                  outputColumnNames: _col0, _col3
-                  Position of Big Table: 1
-                  Select Operator
-                    expressions:
-                          expr: _col0
-                          type: string
-                          expr: _col3
-                          type: string
-                    outputColumnNames: _col0, _col3
-                    Group By Operator
-                      aggregations:
-                            expr: sum(hash(_col0,_col3))
-                      bucketGroup: false
-                      mode: hash
-                      outputColumnNames: _col0
-                      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-11
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        src1:src 
-          Fetch Operator
-            limit: -1
-        src2:src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 80.0) and (key < 100.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                    2 
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                    2 [Column[_col0]]
-                  Position of Big Table: 2
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 100.0) and (key < 80.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                    2 
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                    2 [Column[_col0]]
-                  Position of Big Table: 2
-
-  Stage: Stage-7
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src3:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 100.0) and (key < 80.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                       Inner Join 0 to 2
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col1}
-                    2 
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                    2 [Column[_col0]]
-                  outputColumnNames: _col0, _col3
-                  Position of Big Table: 2
-                  Select Operator
-                    expressions:
-                          expr: _col0
-                          type: string
-                          expr: _col3
-                          type: string
-                    outputColumnNames: _col0, _col3
-                    Group By Operator
-                      aggregations:
-                            expr: sum(hash(_col0,_col3))
-                      bucketGroup: false
-                      mode: hash
-                      outputColumnNames: _col0
-                      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-1
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 80.0) and (key < 100.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                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
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 100.0) and (key < 80.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                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: string
-        src3:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: ((key < 100.0) and (key < 80.0))
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                Reduce Output Operator
-                  key expressions:
-                        expr: _col0
-                        type: string
-                  sort order: +
-                  Map-reduce partition columns:
-                        expr: _col0
-                        type: string
-                  tag: 2
-      Reduce Operator Tree:
-        Join Operator
-          condition map:
-               Inner Join 0 to 1
-               Inner Join 0 to 2
-          condition expressions:
-            0 {VALUE._col0}
-            1 {VALUE._col1}
-            2 
-          handleSkewJoin: false
-          outputColumnNames: _col0, _col3
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: string
-                  expr: _col3
-                  type: string
-            outputColumnNames: _col0, _col3
-            Group By Operator
-              aggregations:
-                    expr: sum(hash(_col0,_col3))
-              bucketGroup: false
-              mode: hash
-              outputColumnNames: _col0
-              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-0
     Fetch Operator
       limit: -1

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join13.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join13.q.out?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join13.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join13.q.out Wed Mar 20 16:02:08 2013
@@ -24,59 +24,22 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_JOIN (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c1) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c2)))) src1) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c3) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c4)))) src2) (AND (= (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src2) c3)) (< (. (TOK_TABLE_OR_COL src1) c1) 100))) (TOK_SUBQUERY (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key) c5) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value) c6)))) src3) (AND (= (+ (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src2) c3)) (. (TOK_TABLE_OR_COL src3) c5)) (< (. (TOK_
 TABLE_OR_COL src3) c5) 200)))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION sum (TOK_FUNCTION hash (. (TOK_TABLE_OR_COL src1) c1) (. (TOK_TABLE_OR_COL src2) c4)))))))
 
 STAGE DEPENDENCIES:
-  Stage-11 is a root stage , consists of Stage-14, Stage-15, Stage-1
-  Stage-14 has a backup stage: Stage-1
-  Stage-9 depends on stages: Stage-14
-  Stage-8 depends on stages: Stage-1, Stage-9, Stage-10 , consists of Stage-12, Stage-13, Stage-2
-  Stage-12 has a backup stage: Stage-2
-  Stage-6 depends on stages: Stage-12
-  Stage-3 depends on stages: Stage-2, Stage-6, Stage-7
-  Stage-13 has a backup stage: Stage-2
-  Stage-7 depends on stages: Stage-13
-  Stage-2
-  Stage-15 has a backup stage: Stage-1
-  Stage-10 depends on stages: Stage-15
-  Stage-1
+  Stage-8 is a root stage
+  Stage-7 depends on stages: Stage-8
+  Stage-3 depends on stages: Stage-7
   Stage-0 is a root stage
 
 STAGE PLANS:
-  Stage: Stage-11
-    Conditional Operator
-
-  Stage: Stage-14
+  Stage: Stage-8
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        src2:src 
+        src1:src 
+          Fetch Operator
+            limit: -1
+        src3:src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col0} {_col1}
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                  Position of Big Table: 0
-
-  Stage: Stage-9
-    Map Reduce
-      Alias -> Map Operator Tree:
         src1:src 
           TableScan
             alias: src
@@ -89,9 +52,7 @@ STAGE PLANS:
                       expr: key
                       type: string
                 outputColumnNames: _col0
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
+                HashTable Sink Operator
                   condition expressions:
                     0 {_col0}
                     1 {_col0} {_col1}
@@ -99,27 +60,7 @@ STAGE PLANS:
                   keys:
                     0 [Column[_col0]]
                     1 [Column[_col0]]
-                  outputColumnNames: _col0, _col2, _col3
-                  Position of Big Table: 0
-                  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-8
-    Conditional Operator
-
-  Stage: Stage-12
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        src3:src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
+                  Position of Big Table: 1
         src3:src 
           TableScan
             alias: src
@@ -142,41 +83,66 @@ STAGE PLANS:
                     1 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0]()]
                   Position of Big Table: 0
 
-  Stage: Stage-6
+  Stage: Stage-7
     Map Reduce
       Alias -> Map Operator Tree:
-        $INTNAME 
-            Map Join Operator
-              condition map:
-                   Inner Join 0 to 1
-              condition expressions:
-                0 {_col3} {_col0}
-                1 
-              handleSkewJoin: false
-              keys:
-                0 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0], Column[_col2]()]
-                1 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0]()]
-              outputColumnNames: _col1, _col2
-              Position of Big Table: 0
+        src2:src 
+          TableScan
+            alias: src
+            Filter Operator
+              predicate:
+                  expr: (key < 100.0)
+                  type: boolean
               Select Operator
                 expressions:
-                      expr: _col2
+                      expr: key
                       type: string
-                      expr: _col1
+                      expr: value
                       type: string
-                outputColumnNames: _col2, _col1
-                Group By Operator
-                  aggregations:
-                        expr: sum(hash(_col2,_col1))
-                  bucketGroup: false
-                  mode: hash
-                  outputColumnNames: _col0
-                  File Output Operator
-                    compressed: false
-                    GlobalTableId: 0
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                outputColumnNames: _col0, _col1
+                Map Join Operator
+                  condition map:
+                       Inner Join 0 to 1
+                  condition expressions:
+                    0 {_col0}
+                    1 {_col0} {_col1}
+                  handleSkewJoin: false
+                  keys:
+                    0 [Column[_col0]]
+                    1 [Column[_col0]]
+                  outputColumnNames: _col0, _col2, _col3
+                  Position of Big Table: 1
+                    Map Join Operator
+                      condition map:
+                           Inner Join 0 to 1
+                      condition expressions:
+                        0 {_col3} {_col0}
+                        1 
+                      handleSkewJoin: false
+                      keys:
+                        0 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0], Column[_col2]()]
+                        1 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0]()]
+                      outputColumnNames: _col1, _col2
+                      Position of Big Table: 0
+                      Select Operator
+                        expressions:
+                              expr: _col2
+                              type: string
+                              expr: _col1
+                              type: string
+                        outputColumnNames: _col2, _col1
+                        Group By Operator
+                          aggregations:
+                                expr: sum(hash(_col2,_col1))
+                          bucketGroup: false
+                          mode: hash
+                          outputColumnNames: _col0
+                          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
 
@@ -209,279 +175,6 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
 
-  Stage: Stage-13
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        $INTNAME 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        $INTNAME 
-            HashTable Sink Operator
-              condition expressions:
-                0 {_col3} {_col0}
-                1 
-              handleSkewJoin: false
-              keys:
-                0 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0], Column[_col2]()]
-                1 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0]()]
-              Position of Big Table: 1
-
-  Stage: Stage-7
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src3:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 200.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  condition expressions:
-                    0 {_col3} {_col0}
-                    1 
-                  handleSkewJoin: false
-                  keys:
-                    0 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0], Column[_col2]()]
-                    1 [class org.apache.hadoop.hive.ql.udf.generic.GenericUDFBridge(Column[_col0]()]
-                  outputColumnNames: _col1, _col2
-                  Position of Big Table: 1
-                  Select Operator
-                    expressions:
-                          expr: _col2
-                          type: string
-                          expr: _col1
-                          type: string
-                    outputColumnNames: _col2, _col1
-                    Group By Operator
-                      aggregations:
-                            expr: sum(hash(_col2,_col1))
-                      bucketGroup: false
-                      mode: hash
-                      outputColumnNames: _col0
-                      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 + _col2)
-                    type: double
-              sort order: +
-              Map-reduce partition columns:
-                    expr: (_col0 + _col2)
-                    type: double
-              tag: 0
-              value expressions:
-                    expr: _col3
-                    type: string
-                    expr: _col0
-                    type: string
-        src3:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 200.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                Reduce Output Operator
-                  key expressions:
-                        expr: UDFToDouble(_col0)
-                        type: double
-                  sort order: +
-                  Map-reduce partition columns:
-                        expr: UDFToDouble(_col0)
-                        type: double
-                  tag: 1
-      Reduce Operator Tree:
-        Join Operator
-          condition map:
-               Inner Join 0 to 1
-          condition expressions:
-            0 {VALUE._col1} {VALUE._col2}
-            1 
-          handleSkewJoin: false
-          outputColumnNames: _col1, _col2
-          Select Operator
-            expressions:
-                  expr: _col2
-                  type: string
-                  expr: _col1
-                  type: string
-            outputColumnNames: _col2, _col1
-            Group By Operator
-              aggregations:
-                    expr: sum(hash(_col2,_col1))
-              bucketGroup: false
-              mode: hash
-              outputColumnNames: _col0
-              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-15
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        src1:src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                HashTable Sink Operator
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col0} {_col1}
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                  Position of Big Table: 1
-
-  Stage: Stage-10
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                outputColumnNames: _col0, _col1
-                Map Join Operator
-                  condition map:
-                       Inner Join 0 to 1
-                  condition expressions:
-                    0 {_col0}
-                    1 {_col0} {_col1}
-                  handleSkewJoin: false
-                  keys:
-                    0 [Column[_col0]]
-                    1 [Column[_col0]]
-                  outputColumnNames: _col0, _col2, _col3
-                  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-1
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src1:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                outputColumnNames: _col0
-                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
-        src2:src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key < 100.0)
-                  type: boolean
-              Select Operator
-                expressions:
-                      expr: key
-                      type: string
-                      expr: value
-                      type: string
-                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: _col0
-                        type: string
-                        expr: _col1
-                        type: string
-      Reduce Operator Tree:
-        Join Operator
-          condition map:
-               Inner Join 0 to 1
-          condition expressions:
-            0 {VALUE._col0}
-            1 {VALUE._col0} {VALUE._col1}
-          handleSkewJoin: false
-          outputColumnNames: _col0, _col2, _col3
-          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-0
     Fetch Operator
       limit: -1

Modified: hive/trunk/ql/src/test/results/clientpositive/auto_join14_hadoop20.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/auto_join14_hadoop20.q.out?rev=1458903&r1=1458902&r2=1458903&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/auto_join14_hadoop20.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/auto_join14_hadoop20.q.out Wed Mar 20 16:02:08 2013
@@ -19,29 +19,22 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME src)) (TOK_TABREF (TOK_TABNAME srcpart)) (and (AND (= (. (TOK_TABLE_OR_COL src) key) (. (TOK_TABLE_OR_COL srcpart) key)) (= (. (TOK_TABLE_OR_COL srcpart) ds) '2008-04-08')) (> (. (TOK_TABLE_OR_COL src) key) 100)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME dest1))) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL srcpart) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-6 is a root stage , consists of Stage-7, Stage-8, Stage-1
-  Stage-7 has a backup stage: Stage-1
-  Stage-4 depends on stages: Stage-7
-  Stage-0 depends on stages: Stage-1, Stage-4, Stage-5
+  Stage-5 is a root stage
+  Stage-4 depends on stages: Stage-5
+  Stage-0 depends on stages: Stage-4
   Stage-2 depends on stages: Stage-0
-  Stage-8 has a backup stage: Stage-1
-  Stage-5 depends on stages: Stage-8
-  Stage-1
 
 STAGE PLANS:
-  Stage: Stage-6
-    Conditional Operator
-
-  Stage: Stage-7
+  Stage: Stage-5
     Map Reduce Local Work
       Alias -> Map Local Tables:
-        srcpart 
+        src 
           Fetch Operator
             limit: -1
       Alias -> Map Local Operator Tree:
-        srcpart 
+        src 
           TableScan
-            alias: srcpart
+            alias: src
             Filter Operator
               predicate:
                   expr: (key > 100.0)
@@ -54,14 +47,14 @@ STAGE PLANS:
                 keys:
                   0 [Column[key]]
                   1 [Column[key]]
-                Position of Big Table: 0
+                Position of Big Table: 1
 
   Stage: Stage-4
     Map Reduce
       Alias -> Map Operator Tree:
-        src 
+        srcpart 
           TableScan
-            alias: src
+            alias: srcpart
             Filter Operator
               predicate:
                   expr: (key > 100.0)
@@ -77,7 +70,7 @@ STAGE PLANS:
                   0 [Column[key]]
                   1 [Column[key]]
                 outputColumnNames: _col0, _col5
-                Position of Big Table: 0
+                Position of Big Table: 1
                 Select Operator
                   expressions:
                         expr: UDFToInteger(_col0)
@@ -109,136 +102,6 @@ STAGE PLANS:
   Stage: Stage-2
     Stats-Aggr Operator
 
-  Stage: Stage-8
-    Map Reduce Local Work
-      Alias -> Map Local Tables:
-        src 
-          Fetch Operator
-            limit: -1
-      Alias -> Map Local Operator Tree:
-        src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key > 100.0)
-                  type: boolean
-              HashTable Sink Operator
-                condition expressions:
-                  0 {key}
-                  1 {value}
-                handleSkewJoin: false
-                keys:
-                  0 [Column[key]]
-                  1 [Column[key]]
-                Position of Big Table: 1
-
-  Stage: Stage-5
-    Map Reduce
-      Alias -> Map Operator Tree:
-        srcpart 
-          TableScan
-            alias: srcpart
-            Filter Operator
-              predicate:
-                  expr: (key > 100.0)
-                  type: boolean
-              Map Join Operator
-                condition map:
-                     Inner Join 0 to 1
-                condition expressions:
-                  0 {key}
-                  1 {value}
-                handleSkewJoin: false
-                keys:
-                  0 [Column[key]]
-                  1 [Column[key]]
-                outputColumnNames: _col0, _col5
-                Position of Big Table: 1
-                Select Operator
-                  expressions:
-                        expr: UDFToInteger(_col0)
-                        type: int
-                        expr: _col5
-                        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.dest1
-      Local Work:
-        Map Reduce Local Work
-
-  Stage: Stage-1
-    Map Reduce
-      Alias -> Map Operator Tree:
-        src 
-          TableScan
-            alias: src
-            Filter Operator
-              predicate:
-                  expr: (key > 100.0)
-                  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
-        srcpart 
-          TableScan
-            alias: srcpart
-            Filter Operator
-              predicate:
-                  expr: (key > 100.0)
-                  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: value
-                      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: UDFToInteger(_col0)
-                  type: int
-                  expr: _col5
-                  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.dest1
-
 
 PREHOOK: query: FROM src JOIN srcpart ON src.key = srcpart.key AND srcpart.ds = '2008-04-08' and src.key > 100
 INSERT OVERWRITE TABLE dest1 SELECT src.key, srcpart.value