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 2012/08/04 19:08:51 UTC

svn commit: r1369375 [1/7] - in /hive/trunk: contrib/src/test/results/clientpositive/ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ ql/src/java/org/apache/hadoop/hive/ql/plan/ ql/src/test/queries/clientpositive/ ql/src/test/results/clientpositive/ q...

Author: namit
Date: Sat Aug  4 17:08:47 2012
New Revision: 1369375

URL: http://svn.apache.org/viewvc?rev=1369375&view=rev
Log:
HIVE-3302 Race condition in query plan for merging at the end of a query
(Kevin Wilfong via namit)


Added:
    hive/trunk/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
    hive/trunk/ql/src/test/results/clientpositive/merge_dynamic_partition5.q.out
Modified:
    hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes.q.out
    hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out
    hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out
    hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java
    hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin2.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin3.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin4.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
    hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
    hive/trunk/ql/src/test/results/clientpositive/case_sensitivity.q.out
    hive/trunk/ql/src/test/results/clientpositive/cast1.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_file_format.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_mult_tables_compact.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_multiple.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_partitioned.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_auto_update.q.out
    hive/trunk/ql/src/test/results/clientpositive/index_compression.q.out
    hive/trunk/ql/src/test/results/clientpositive/input11.q.out
    hive/trunk/ql/src/test/results/clientpositive/input12.q.out
    hive/trunk/ql/src/test/results/clientpositive/input13.q.out
    hive/trunk/ql/src/test/results/clientpositive/input34.q.out
    hive/trunk/ql/src/test/results/clientpositive/input35.q.out
    hive/trunk/ql/src/test/results/clientpositive/input36.q.out
    hive/trunk/ql/src/test/results/clientpositive/input38.q.out
    hive/trunk/ql/src/test/results/clientpositive/input6.q.out
    hive/trunk/ql/src/test/results/clientpositive/input7.q.out
    hive/trunk/ql/src/test/results/clientpositive/input8.q.out
    hive/trunk/ql/src/test/results/clientpositive/input9.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_dynamicserde.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_part1.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_part2.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_part5.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_testsequencefile.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_testxpath.q.out
    hive/trunk/ql/src/test/results/clientpositive/input_testxpath2.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert1.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_into4.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_into5.q.out
    hive/trunk/ql/src/test/results/clientpositive/insert_into6.q.out
    hive/trunk/ql/src/test/results/clientpositive/join25.q.out
    hive/trunk/ql/src/test/results/clientpositive/join26.q.out
    hive/trunk/ql/src/test/results/clientpositive/join27.q.out
    hive/trunk/ql/src/test/results/clientpositive/join28.q.out
    hive/trunk/ql/src/test/results/clientpositive/join29.q.out
    hive/trunk/ql/src/test/results/clientpositive/join32.q.out
    hive/trunk/ql/src/test/results/clientpositive/join34.q.out
    hive/trunk/ql/src/test/results/clientpositive/join35.q.out
    hive/trunk/ql/src/test/results/clientpositive/join36.q.out
    hive/trunk/ql/src/test/results/clientpositive/join37.q.out
    hive/trunk/ql/src/test/results/clientpositive/join39.q.out
    hive/trunk/ql/src/test/results/clientpositive/join_map_ppr.q.out
    hive/trunk/ql/src/test/results/clientpositive/lineage1.q.out
    hive/trunk/ql/src/test/results/clientpositive/load_dyn_part1.q.out
    hive/trunk/ql/src/test/results/clientpositive/load_dyn_part14.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge1.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge2.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge3.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge4.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge_dynamic_partition.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge_dynamic_partition2.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge_dynamic_partition3.q.out
    hive/trunk/ql/src/test/results/clientpositive/merge_dynamic_partition4.q.out
    hive/trunk/ql/src/test/results/clientpositive/multi_insert.q.out
    hive/trunk/ql/src/test/results/clientpositive/multi_insert_move_tasks_share_dependencies.q.out
    hive/trunk/ql/src/test/results/clientpositive/pcr.q.out
    hive/trunk/ql/src/test/results/clientpositive/ppd_constant_expr.q.out
    hive/trunk/ql/src/test/results/clientpositive/quote1.q.out
    hive/trunk/ql/src/test/results/clientpositive/rand_partitionpruner2.q.out
    hive/trunk/ql/src/test/results/clientpositive/rcfile_createas1.q.out
    hive/trunk/ql/src/test/results/clientpositive/rcfile_merge1.q.out
    hive/trunk/ql/src/test/results/clientpositive/rcfile_merge2.q.out
    hive/trunk/ql/src/test/results/clientpositive/rcfile_merge3.q.out
    hive/trunk/ql/src/test/results/clientpositive/rcfile_merge4.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample1.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample2.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample4.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample5.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample6.q.out
    hive/trunk/ql/src/test/results/clientpositive/sample7.q.out
    hive/trunk/ql/src/test/results/clientpositive/stats0.q.out
    hive/trunk/ql/src/test/results/clientpositive/stats11.q.out
    hive/trunk/ql/src/test/results/clientpositive/stats4.q.out
    hive/trunk/ql/src/test/results/clientpositive/subq.q.out
    hive/trunk/ql/src/test/results/clientpositive/udf1.q.out
    hive/trunk/ql/src/test/results/clientpositive/udf_10_trims.q.out
    hive/trunk/ql/src/test/results/clientpositive/udf_length.q.out
    hive/trunk/ql/src/test/results/clientpositive/udf_reverse.q.out
    hive/trunk/ql/src/test/results/clientpositive/union.q.out
    hive/trunk/ql/src/test/results/clientpositive/union10.q.out
    hive/trunk/ql/src/test/results/clientpositive/union12.q.out
    hive/trunk/ql/src/test/results/clientpositive/union18.q.out
    hive/trunk/ql/src/test/results/clientpositive/union28.q.out
    hive/trunk/ql/src/test/results/clientpositive/union29.q.out
    hive/trunk/ql/src/test/results/clientpositive/union30.q.out
    hive/trunk/ql/src/test/results/clientpositive/union4.q.out
    hive/trunk/ql/src/test/results/clientpositive/union6.q.out
    hive/trunk/ql/src/test/results/compiler/plan/case_sensitivity.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/input1.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/input2.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/input3.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/input6.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/input7.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/input9.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/input_testsequencefile.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/sample2.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/sample3.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/sample4.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/sample5.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/sample6.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/sample7.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/subq.q.xml
    hive/trunk/ql/src/test/results/compiler/plan/union.q.xml

Modified: hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes.q.out (original)
+++ hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes.q.out Sat Aug  4 17:08:47 2012
@@ -34,11 +34,13 @@ ABSTRACT SYNTAX TREE:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -82,7 +84,7 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                           name: default.dest1
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -117,6 +119,25 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.dest1
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.dest1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: FROM (
   FROM src

Modified: hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out (original)
+++ hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out Sat Aug  4 17:08:47 2012
@@ -34,11 +34,13 @@ ABSTRACT SYNTAX TREE:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -75,7 +77,7 @@ STAGE PLANS:
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                         name: default.dest1
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -110,6 +112,25 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.dest1
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.dest1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: FROM (
   FROM src

Modified: hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out (original)
+++ hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out Sat Aug  4 17:08:47 2012
@@ -34,11 +34,13 @@ ABSTRACT SYNTAX TREE:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -75,7 +77,7 @@ STAGE PLANS:
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                         name: default.dest1
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -110,6 +112,25 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.dest1
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.dest1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: FROM (
   FROM src

Modified: hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out (original)
+++ hive/trunk/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out Sat Aug  4 17:08:47 2012
@@ -34,11 +34,13 @@ ABSTRACT SYNTAX TREE:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -82,7 +84,7 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                           name: default.dest1
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -117,6 +119,25 @@ STAGE PLANS:
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   name: default.dest1
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.dest1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: FROM (
   FROM src

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java Sat Aug  4 17:08:47 2012
@@ -266,7 +266,7 @@ public class GenMRFileSink1 implements N
     ConditionalTask cndTsk = createCondTask(conf, currTask, dummyMv, cplan,
         fsConf.getDirName());
 
-    LinkMoveTask(ctx, newOutput, cndTsk);
+    linkMoveTask(ctx, newOutput, cndTsk);
   }
 
   /**
@@ -425,17 +425,46 @@ public class GenMRFileSink1 implements N
     //
     // 3. add the moveTask as the children of the conditional task
     //
-    LinkMoveTask(ctx, fsOutput, cndTsk);
+    linkMoveTask(ctx, fsOutput, cndTsk);
  }
 
-  private void LinkMoveTask(GenMRProcContext ctx, FileSinkOperator newOutput,
+  /**
+   * Make the move task in the GenMRProcContext following the FileSinkOperator a dependent of all
+   * possible subtrees branching from the ConditionalTask.
+   *
+   * @param ctx
+   * @param newOutput
+   * @param cndTsk
+   */
+  private void linkMoveTask(GenMRProcContext ctx, FileSinkOperator newOutput,
       ConditionalTask cndTsk) {
 
     List<Task<MoveWork>> mvTasks = ctx.getMvTask();
     Task<MoveWork> mvTask = findMoveTask(mvTasks, newOutput);
 
     for (Task<? extends Serializable> tsk : cndTsk.getListTasks()) {
-      addDependentMoveTasks(ctx, mvTask, tsk);
+      linkMoveTask(ctx, mvTask, tsk);
+    }
+  }
+
+  /**
+   * Follows the task tree down from task and makes all leaves parents of mvTask
+   *
+   * @param ctx
+   * @param mvTask
+   * @param task
+   */
+  private void linkMoveTask(GenMRProcContext ctx, Task<MoveWork> mvTask,
+      Task<? extends Serializable> task) {
+
+    if (task.getDependentTasks() == null || task.getDependentTasks().isEmpty()) {
+      // If it's a leaf, add the move task as a child
+      addDependentMoveTasks(ctx, mvTask, task);
+    } else {
+      // Otherwise, for each child run this method recursively
+      for (Task<? extends Serializable> childTask : task.getDependentTasks()) {
+        linkMoveTask(ctx, mvTask, childTask);
+      }
     }
   }
 
@@ -552,8 +581,22 @@ public class GenMRFileSink1 implements N
       Task<? extends Serializable> currTask, MoveWork mvWork,
       MapredWork mergeWork, String inputPath) {
 
-    Task<? extends Serializable> mergeTask = TaskFactory.get(mergeWork, conf);
-    Task<? extends Serializable> moveTask = TaskFactory.get(mvWork, conf);
+    // There are 3 options for this ConditionalTask:
+    // 1) Merge the partitions
+    // 2) Move the partitions (i.e. don't merge the partitions)
+    // 3) Merge some partitions and move other partitions (i.e. merge some partitions and don't
+    //    merge others) in this case the merge is done first followed by the move to prevent
+    //    conflicts.
+    Task<? extends Serializable> mergeOnlyMergeTask = TaskFactory.get(mergeWork, conf);
+    Task<? extends Serializable> moveOnlyMoveTask = TaskFactory.get(mvWork, conf);
+    Task<? extends Serializable> mergeAndMoveMergeTask = TaskFactory.get(mergeWork, conf);
+    Task<? extends Serializable> mergeAndMoveMoveTask = TaskFactory.get(mvWork, conf);
+
+    // NOTE! It is necessary merge task is the parent of the move task, and not
+    // the other way around, for the proper execution of the execute method of
+    // ConditionalTask
+    mergeAndMoveMergeTask.addDependentTask(mergeAndMoveMoveTask);
+
     List<Serializable> listWorks = new ArrayList<Serializable>();
     listWorks.add(mvWork);
     listWorks.add(mergeWork);
@@ -561,8 +604,9 @@ public class GenMRFileSink1 implements N
     ConditionalWork cndWork = new ConditionalWork(listWorks);
 
     List<Task<? extends Serializable>> listTasks = new ArrayList<Task<? extends Serializable>>();
-    listTasks.add(moveTask);
-    listTasks.add(mergeTask);
+    listTasks.add(moveOnlyMoveTask);
+    listTasks.add(mergeOnlyMergeTask);
+    listTasks.add(mergeAndMoveMergeTask);
 
     ConditionalTask cndTsk = (ConditionalTask) TaskFactory.get(cndWork, conf);
     cndTsk.setListTasks(listTasks);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/ConditionalResolverMergeFiles.java Sat Aug  4 17:08:47 2012
@@ -21,13 +21,10 @@ package org.apache.hadoop.hive.ql.plan;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -123,6 +120,7 @@ public class ConditionalResolverMergeFil
 
     Task<? extends Serializable> mvTask = ctx.getListTasks().get(0);
     Task<? extends Serializable> mrTask = ctx.getListTasks().get(1);
+    Task<? extends Serializable> mrAndMvTask = ctx.getListTasks().get(2);
 
     try {
       Path dirPath = new Path(dirName);
@@ -179,11 +177,17 @@ public class ConditionalResolverMergeFil
           if (doMerge) {
             // add the merge MR job
             setupMapRedWork(conf, work, trgtSize, totalSz);
-            resTsks.add(mrTask);
 
             // add the move task for those partitions that do not need merging
-          	if (toMove.size() > 0) { //
+            if (toMove.size() > 0) {
           	  // modify the existing move task as it is already in the candidate running tasks
+
+          	  // running the MoveTask and MR task in parallel may
+              // cause the mvTask write to /ds=1 and MR task write
+              // to /ds=1_1 for the same partition.
+              // make the MoveTask as the child of the MR Task
+          	  resTsks.add(mrAndMvTask);
+
           	  MoveWork mvWork = (MoveWork) mvTask.getWork();
           	  LoadFileDesc lfd = mvWork.getLoadFileWork();
 
@@ -212,21 +216,8 @@ public class ConditionalResolverMergeFil
           	  mvWork.setLoadFileWork(null);
           	  mvWork.setLoadTableWork(null);
           	  mvWork.setMultiFilesDesc(lmfd);
-
-          	  // running the MoveTask and MR task in parallel may
-          	  // cause the mvTask write to /ds=1 and MR task write
-          	  // to /ds=1_1 for the same partition.
-          	  // make the MoveTask as the child of the MR Task
-          	  List<Task <? extends Serializable>> cTasks = mrTask.getDependentTasks();
-          	  if (cTasks != null) {
-          	    Iterator<Task <? extends Serializable>> itr = cTasks.iterator();
-          	    while (itr.hasNext()) {
-          	      Task<? extends Serializable> cld = itr.next();
-          	      itr.remove();
-          	      mvTask.addDependentTask(cld);
-          	    }
-          	  }
-          	  mrTask.addDependentTask(mvTask);
+          	} else {
+          	  resTsks.add(mrTask);
           	}
           } else { // add the move task
             resTsks.add(mvTask);
@@ -246,6 +237,10 @@ public class ConditionalResolverMergeFil
     } catch (IOException e) {
       e.printStackTrace();
     }
+
+    // Only one of the tasks should ever be added to resTsks
+    assert(resTsks.size() == 1);
+
     return resTsks;
   }
 

Added: hive/trunk/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q?rev=1369375&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q Sat Aug  4 17:08:47 2012
@@ -0,0 +1,38 @@
+-- this is to test the case where some dynamic partitions are merged and some are moved
+
+create table srcpart_merge_dp like srcpart;
+
+create table srcpart_merge_dp_rc like srcpart;
+alter table srcpart_merge_dp_rc set fileformat RCFILE;
+
+create table merge_dynamic_part like srcpart;
+alter table merge_dynamic_part set fileformat RCFILE;
+
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+
+load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+
+insert overwrite table srcpart_merge_dp_rc partition (ds = '2008-04-08', hr) 
+select key, value, hr from srcpart_merge_dp where ds = '2008-04-08';
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
+set hive.merge.mapfiles=true;
+set hive.merge.mapredfiles=true;
+set hive.merge.smallfiles.avgsize=200;
+set hive.exec.compress.output=false;
+set hive.exec.dynamic.partition=true;
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+explain
+insert overwrite table merge_dynamic_part partition (ds = '2008-04-08', hr)
+select key, value, if(key % 100 == 0, 'a1', 'b1') as hr from srcpart_merge_dp_rc where ds = '2008-04-08';
+
+insert overwrite table merge_dynamic_part partition (ds = '2008-04-08', hr)
+select key, value, if(key % 100 == 0, 'a1', 'b1') as hr from srcpart_merge_dp_rc where ds = '2008-04-08';
+
+show partitions merge_dynamic_part;
+
+select count(*) from merge_dynamic_part;

Modified: hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/binary_output_format.q.out Sat Aug  4 17:08:47 2012
@@ -54,11 +54,13 @@ ABSTRACT SYNTAX TREE:
 
 STAGE DEPENDENCIES:
   Stage-1 is a root stage
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
   Stage: Stage-1
@@ -148,7 +150,7 @@ STAGE PLANS:
               name: default.src
             name: default.src
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -251,6 +253,79 @@ STAGE PLANS:
               name: default.dest1
             name: default.dest1
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns mydata
+                    columns.types string
+#### A masked pattern was here ####
+                    name default.dest1
+                    serialization.ddl struct dest1 { string mydata}
+                    serialization.format 1
+                    serialization.last.column.takes.rest true
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.dest1
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveBinaryOutputFormat
+            properties:
+              bucket_count -1
+              columns mydata
+              columns.types string
+#### A masked pattern was here ####
+              name default.dest1
+              serialization.ddl struct dest1 { string mydata}
+              serialization.format 1
+              serialization.last.column.takes.rest true
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### 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.HiveBinaryOutputFormat
+              properties:
+                bucket_count -1
+                columns mydata
+                columns.types string
+#### A masked pattern was here ####
+                name default.dest1
+                serialization.ddl struct dest1 { string mydata}
+                serialization.format 1
+                serialization.last.column.takes.rest true
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.dest1
+            name: default.dest1
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: INSERT OVERWRITE TABLE dest1
 SELECT TRANSFORM(*)

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin1.q.out Sat Aug  4 17:08:47 2012
@@ -346,16 +346,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08"))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         b 
@@ -487,7 +489,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin
             name: default.srcbucket_mapjoin
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -586,6 +588,76 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    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.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              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
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns key,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                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.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result 
 select /*+mapjoin(b)*/ a.key, a.value, b.value 
@@ -754,16 +826,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value))) (TOK_WHERE (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08"))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         a 
@@ -904,7 +978,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin_part
             name: default.srcbucket_mapjoin_part
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -1023,6 +1097,91 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    numFiles 1
+                    numPartitions 0
+                    numRows 464
+                    rawDataSize 8519
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 8983
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              numFiles 1
+              numPartitions 0
+              numRows 464
+              rawDataSize 8519
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 8983
+#### 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,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                numFiles 1
+                numPartitions 0
+                numRows 464
+                rawDataSize 8519
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 8983
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result 
 select /*+mapjoin(a)*/ a.key, a.value, b.value 

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin2.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin2.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin2.q.out Sat Aug  4 17:08:47 2012
@@ -77,16 +77,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         b 
@@ -220,7 +222,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin_part
             name: default.srcbucket_mapjoin_part
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -319,6 +321,76 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    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.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              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
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns key,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                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.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result 
 select /*+mapjoin(b)*/ a.key, a.value, b.value 
@@ -487,16 +559,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         a 
@@ -635,7 +709,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin_part_2
             name: default.srcbucket_mapjoin_part_2
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -754,6 +828,91 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    numFiles 1
+                    numPartitions 0
+                    numRows 564
+                    rawDataSize 10503
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 11067
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              numFiles 1
+              numPartitions 0
+              numRows 564
+              rawDataSize 10503
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 11067
+#### 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,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                numFiles 1
+                numPartitions 0
+                numRows 564
+                rawDataSize 10503
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 11067
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result 
 select /*+mapjoin(a)*/ a.key, a.value, b.value 
@@ -1081,16 +1240,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         b 
@@ -1229,7 +1390,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin_part
             name: default.srcbucket_mapjoin_part
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -1348,6 +1509,91 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    numFiles 1
+                    numPartitions 0
+                    numRows 564
+                    rawDataSize 10503
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 11067
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              numFiles 1
+              numPartitions 0
+              numRows 564
+              rawDataSize 10503
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 11067
+#### 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,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                numFiles 1
+                numPartitions 0
+                numRows 564
+                rawDataSize 10503
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 11067
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result
 select /*+mapjoin(b)*/ a.key, a.value, b.value

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin3.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin3.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin3.q.out Sat Aug  4 17:08:47 2012
@@ -94,16 +94,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (and (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")) (= (. (TOK_TABLE_OR_COL a) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         b 
@@ -237,7 +239,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin_part_2
             name: default.srcbucket_mapjoin_part_2
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -336,6 +338,76 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    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.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              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
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns key,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                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.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result 
 select /*+mapjoin(b)*/ a.key, a.value, b.value 
@@ -504,16 +576,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part_2) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin_part) b) (and (and (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)) (= (. (TOK_TABLE_OR_COL b) ds) "2008-04-08")) (= (. (TOK_TABLE_OR_COL a) ds) "2008-04-08")))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         a 
@@ -652,7 +726,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin_part
             name: default.srcbucket_mapjoin_part
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -771,6 +845,91 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    numFiles 1
+                    numPartitions 0
+                    numRows 564
+                    rawDataSize 10503
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 11067
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              numFiles 1
+              numPartitions 0
+              numRows 564
+              rawDataSize 10503
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 11067
+#### 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,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                numFiles 1
+                numPartitions 0
+                numRows 564
+                rawDataSize 10503
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 11067
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result 
 select /*+mapjoin(a)*/ a.key, a.value, b.value 

Modified: hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin4.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin4.q.out?rev=1369375&r1=1369374&r2=1369375&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin4.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/bucketmapjoin4.q.out Sat Aug  4 17:08:47 2012
@@ -94,16 +94,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST b))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         b 
@@ -233,7 +235,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin
             name: default.srcbucket_mapjoin
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -332,6 +334,76 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    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.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              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
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                columns key,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                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.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result
 select /*+mapjoin(b)*/ a.key, a.value, b.value
@@ -496,16 +568,18 @@ ABSTRACT SYNTAX TREE:
   (TOK_QUERY (TOK_FROM (TOK_JOIN (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) a) (TOK_TABREF (TOK_TABNAME srcbucket_mapjoin) b) (= (. (TOK_TABLE_OR_COL a) key) (. (TOK_TABLE_OR_COL b) key)))) (TOK_INSERT (TOK_DESTINATION (TOK_TAB (TOK_TABNAME bucketmapjoin_tmp_result))) (TOK_SELECT (TOK_HINTLIST (TOK_HINT TOK_MAPJOIN (TOK_HINTARGLIST a))) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL a) value)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL b) value)))))
 
 STAGE DEPENDENCIES:
-  Stage-7 is a root stage
-  Stage-1 depends on stages: Stage-7
-  Stage-5 depends on stages: Stage-1 , consists of Stage-4, Stage-3
+  Stage-9 is a root stage
+  Stage-1 depends on stages: Stage-9
+  Stage-7 depends on stages: Stage-1 , consists of Stage-4, Stage-3, Stage-5
   Stage-4
-  Stage-0 depends on stages: Stage-4, Stage-3
+  Stage-0 depends on stages: Stage-4, Stage-3, Stage-6
   Stage-2 depends on stages: Stage-0
   Stage-3
+  Stage-5
+  Stage-6 depends on stages: Stage-5
 
 STAGE PLANS:
-  Stage: Stage-7
+  Stage: Stage-9
     Map Reduce Local Work
       Alias -> Map Local Tables:
         a 
@@ -640,7 +714,7 @@ STAGE PLANS:
               name: default.srcbucket_mapjoin
             name: default.srcbucket_mapjoin
 
-  Stage: Stage-5
+  Stage: Stage-7
     Conditional Operator
 
   Stage: Stage-4
@@ -759,6 +833,91 @@ STAGE PLANS:
               name: default.bucketmapjoin_tmp_result
             name: default.bucketmapjoin_tmp_result
 
+  Stage: Stage-5
+    Map Reduce
+      Alias -> Map Operator Tree:
+#### A masked pattern was here ####
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    bucket_count -1
+                    columns key,value1,value2
+                    columns.types string:string:string
+#### A masked pattern was here ####
+                    name default.bucketmapjoin_tmp_result
+                    numFiles 1
+                    numPartitions 0
+                    numRows 464
+                    rawDataSize 8519
+                    serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 8983
+#### A masked pattern was here ####
+                  serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  name: default.bucketmapjoin_tmp_result
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
+      Needs Tagging: false
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -ext-10002
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            properties:
+              bucket_count -1
+              columns key,value1,value2
+              columns.types string:string:string
+#### A masked pattern was here ####
+              name default.bucketmapjoin_tmp_result
+              numFiles 1
+              numPartitions 0
+              numRows 464
+              rawDataSize 8519
+              serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 8983
+#### 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,value1,value2
+                columns.types string:string:string
+#### A masked pattern was here ####
+                name default.bucketmapjoin_tmp_result
+                numFiles 1
+                numPartitions 0
+                numRows 464
+                rawDataSize 8519
+                serialization.ddl struct bucketmapjoin_tmp_result { string key, string value1, string value2}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                totalSize 8983
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.bucketmapjoin_tmp_result
+            name: default.bucketmapjoin_tmp_result
+
+  Stage: Stage-6
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
 
 PREHOOK: query: insert overwrite table bucketmapjoin_tmp_result
 select /*+mapjoin(a)*/ a.key, a.value, b.value