You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by su...@apache.org on 2017/08/07 16:33:35 UTC

hive git commit: HIVE-17213: HoS: file merging doesn't work for union all (Chao Sun, reviewed by Xuefu Zhang)

Repository: hive
Updated Branches:
  refs/heads/master fc6725eb4 -> 318822b18


HIVE-17213: HoS: file merging doesn't work for union all (Chao Sun, reviewed by Xuefu Zhang)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/318822b1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/318822b1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/318822b1

Branch: refs/heads/master
Commit: 318822b1818c620378e8cb80c4f201ed69cf4be3
Parents: fc6725e
Author: Chao Sun <su...@apache.org>
Authored: Wed Aug 2 09:39:02 2017 -0700
Committer: Chao Sun <su...@apache.org>
Committed: Mon Aug 7 09:33:08 2017 -0700

----------------------------------------------------------------------
 .../test/resources/testconfiguration.properties |   8 +-
 .../hadoop/hive/cli/control/CliConfigs.java     |   4 +-
 .../ql/parse/spark/GenSparkProcContext.java     |   5 +
 .../hive/ql/parse/spark/GenSparkUtils.java      |  79 ++-
 .../queries/clientpositive/spark_union_merge.q  |  41 ++
 .../spark/spark_union_merge.q.out               | 564 +++++++++++++++++++
 .../ptest2/conf/deployed/master-mr2.properties  |   4 +-
 7 files changed, 699 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/318822b1/itests/src/test/resources/testconfiguration.properties
----------------------------------------------------------------------
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 685b388..772113a 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -1390,8 +1390,12 @@ spark.query.files=add_part_multiple.q, \
   windowing.q
 
 # Unlike "spark.query.files" above, these tests only run
-# under Spark engine.
-spark.only.query.files=spark_combine_equivalent_work.q,\
+# under Spark engine and only use TestSparkCliDriver.
+spark.only.query.files=spark_union_merge.q
+
+# Unlike "miniSparkOnYarn.query.files" below, these tests only run
+# under Spark engine and only use TestMiniSparkOnYarnCliDriver.
+miniSparkOnYarn.only.query.files=spark_combine_equivalent_work.q,\
   spark_dynamic_partition_pruning.q,\
   spark_dynamic_partition_pruning_2.q,\
   spark_dynamic_partition_pruning_3.q,\

http://git-wip-us.apache.org/repos/asf/hive/blob/318822b1/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
index 27b87fb..569f48f 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/cli/control/CliConfigs.java
@@ -52,6 +52,7 @@ public class CliConfigs {
         excludesFrom(testConfigProps, "minitez.query.files");
         excludesFrom(testConfigProps, "encrypted.query.files");
         excludesFrom(testConfigProps, "spark.only.query.files");
+        excludesFrom(testConfigProps, "miniSparkOnYarn.only.query.files");
         excludesFrom(testConfigProps, "disabled.query.files");
 
         setResultsDir("ql/src/test/results/clientpositive");
@@ -455,6 +456,7 @@ public class CliConfigs {
         setQueryDir("ql/src/test/queries/clientpositive");
 
         includesFrom(testConfigProps, "spark.query.files");
+        includesFrom(testConfigProps, "spark.only.query.files");
 
         setResultsDir("ql/src/test/results/clientpositive/spark");
         setLogDir("itests/qtest-spark/target/qfile-results/clientpositive/spark");
@@ -477,7 +479,7 @@ public class CliConfigs {
         setQueryDir("ql/src/test/queries/clientpositive");
 
         includesFrom(testConfigProps, "miniSparkOnYarn.query.files");
-        includesFrom(testConfigProps, "spark.only.query.files");
+        includesFrom(testConfigProps, "miniSparkOnYarn.only.query.files");
 
         setResultsDir("ql/src/test/results/clientpositive/spark");
         setLogDir("itests/qtest-spark/target/qfile-results/clientpositive/spark");

http://git-wip-us.apache.org/repos/asf/hive/blob/318822b1/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
index b0ab495..4d2bcfa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkProcContext.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.parse.spark;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.ObjectPair;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask;
@@ -38,6 +39,7 @@ import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.DependencyCollectionWork;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.ReduceWork;
@@ -131,6 +133,8 @@ public class GenSparkProcContext implements NodeProcessorCtx {
   public final List<UnionOperator> currentUnionOperators;
   public final Set<BaseWork> workWithUnionOperators;
 
+  // we link filesink that will write to the same final location
+  public final Map<Path, List<FileSinkDesc>> linkedFileSinks;
   public final Set<FileSinkOperator> fileSinkSet;
   public final Map<FileSinkOperator, List<FileSinkOperator>> fileSinkMap;
 
@@ -180,6 +184,7 @@ public class GenSparkProcContext implements NodeProcessorCtx {
     this.unionWorkMap = new LinkedHashMap<Operator<?>, BaseWork>();
     this.currentUnionOperators = new LinkedList<UnionOperator>();
     this.workWithUnionOperators = new LinkedHashSet<BaseWork>();
+    this.linkedFileSinks = new LinkedHashMap<>();
     this.fileSinkSet = new LinkedHashSet<FileSinkOperator>();
     this.fileSinkMap = new LinkedHashMap<FileSinkOperator, List<FileSinkOperator>>();
     this.pruningSinkSet = new LinkedHashSet<Operator<?>>();

http://git-wip-us.apache.org/repos/asf/hive/blob/318822b1/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
index 08602e2..4f826a1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/GenSparkUtils.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.parse.spark;
 
+import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Deque;
 import java.util.HashMap;
@@ -31,6 +32,8 @@ import java.util.Set;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.DependencyCollectionTask;
 import org.apache.hadoop.hive.ql.exec.FetchTask;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
 import org.apache.hadoop.hive.ql.exec.ForwardOperator;
@@ -43,6 +46,7 @@ import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.UnionOperator;
 import org.apache.hadoop.hive.ql.exec.spark.SparkUtilities;
 import org.apache.hadoop.hive.ql.optimizer.GenMapRedUtils;
@@ -53,7 +57,9 @@ import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.BaseWork;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
+import org.apache.hadoop.hive.ql.plan.FileSinkDesc;
 import org.apache.hadoop.hive.ql.plan.MapWork;
+import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.plan.OperatorDesc;
 import org.apache.hadoop.hive.ql.plan.ReduceWork;
 import org.apache.hadoop.hive.ql.plan.SparkEdgeProperty;
@@ -269,6 +275,11 @@ public class GenSparkUtils {
 
     Iterator<Operator<?>> it = newRoots.iterator();
     for (Operator<?> orig: roots) {
+      Set<FileSinkOperator> fsOpSet = OperatorUtils.findOperators(orig, FileSinkOperator.class);
+      for (FileSinkOperator fsOp : fsOpSet) {
+        context.fileSinkSet.remove(fsOp);
+      }
+
       Operator<?> newRoot = it.next();
       if (newRoot instanceof HashTableDummyOperator) {
         dummyOps.add((HashTableDummyOperator) newRoot);
@@ -290,6 +301,26 @@ public class GenSparkUtils {
       Operator<?> current = operators.pop();
       seen.add(current);
 
+      if (current instanceof FileSinkOperator) {
+        FileSinkOperator fileSink = (FileSinkOperator)current;
+
+        // remember it for additional processing later
+        context.fileSinkSet.add(fileSink);
+
+        FileSinkDesc desc = fileSink.getConf();
+        Path path = desc.getDirName();
+        List<FileSinkDesc> linked;
+
+        if (!context.linkedFileSinks.containsKey(path)) {
+          linked = new ArrayList<FileSinkDesc>();
+          context.linkedFileSinks.put(path, linked);
+        }
+        linked = context.linkedFileSinks.get(path);
+        linked.add(desc);
+
+        desc.setLinkedFileSinkDesc(linked);
+      }
+
       if (current instanceof UnionOperator) {
         Operator<?> parent = null;
         int count = 0;
@@ -344,7 +375,7 @@ public class GenSparkUtils {
       }
     }
 
-    Path finalName = GenMapRedUtils.createMoveTask(context.currentTask,
+    Path finalName = createMoveTask(context.currentTask,
         chDir, fileSink, parseContext, context.moveTask, hconf, context.dependencyTask);
 
     if (chDir) {
@@ -365,6 +396,52 @@ public class GenSparkUtils {
   }
 
   /**
+   * Create and add any dependent move tasks.
+   *
+   * This is forked from {@link GenMapRedUtils}. The difference is that it doesn't check
+   * 'isLinkedFileSink' and does not set parent dir for the linked file sinks.
+   */
+  public static Path createMoveTask(Task<? extends Serializable> currTask, boolean chDir,
+      FileSinkOperator fsOp, ParseContext parseCtx, List<Task<MoveWork>> mvTasks,
+      HiveConf hconf, DependencyCollectionTask dependencyTask) {
+
+    Path dest = null;
+
+    if (chDir) {
+      dest = fsOp.getConf().getFinalDirName();
+
+      // generate the temporary file
+      // it must be on the same file system as the current destination
+      Context baseCtx = parseCtx.getContext();
+
+      Path tmpDir = baseCtx.getExternalTmpPath(dest);
+
+      FileSinkDesc fileSinkDesc = fsOp.getConf();
+      // Change all the linked file sink descriptors
+      if (fileSinkDesc.getLinkedFileSinkDesc() != null) {
+        for (FileSinkDesc fsConf : fileSinkDesc.getLinkedFileSinkDesc()) {
+          fsConf.setDirName(tmpDir);
+        }
+      } else {
+        fileSinkDesc.setDirName(tmpDir);
+      }
+    }
+
+    Task<MoveWork> mvTask = null;
+
+    if (!chDir) {
+      mvTask = GenMapRedUtils.findMoveTask(mvTasks, fsOp);
+    }
+
+    // Set the move task to be dependent on the current task
+    if (mvTask != null) {
+      GenMapRedUtils.addDependentMoveTasks(mvTask, hconf, currTask, dependencyTask);
+    }
+
+    return dest;
+  }
+
+  /**
    * Populate partition pruning information from the pruning sink operator to the
    * target MapWork (the MapWork for the big table side). The information include the source table
    * name, column name, and partition key expression. It also set up the temporary path used to

http://git-wip-us.apache.org/repos/asf/hive/blob/318822b1/ql/src/test/queries/clientpositive/spark_union_merge.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/spark_union_merge.q b/ql/src/test/queries/clientpositive/spark_union_merge.q
new file mode 100644
index 0000000..3121078
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/spark_union_merge.q
@@ -0,0 +1,41 @@
+set hive.mapred.mode=nonstrict;
+-- union case: both subqueries are map jobs on same input, followed by filesink
+-- mostly copied from union.q
+
+set hive.merge.sparkfiles=false;
+
+EXPLAIN EXTENDED
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*;
+
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*;
+
+dfs -ls ${system:test.warehouse.dir}/union_merge.out;
+
+set hive.merge.sparkfiles=true;
+
+EXPLAIN EXTENDED
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*;
+
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*;
+
+dfs -ls ${system:test.warehouse.dir}/union_merge.out;

http://git-wip-us.apache.org/repos/asf/hive/blob/318822b1/ql/src/test/results/clientpositive/spark/spark_union_merge.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/spark_union_merge.q.out b/ql/src/test/results/clientpositive/spark/spark_union_merge.q.out
new file mode 100644
index 0000000..f6afd8b
--- /dev/null
+++ b/ql/src/test/results/clientpositive/spark/spark_union_merge.q.out
@@ -0,0 +1,564 @@
+PREHOOK: query: EXPLAIN EXTENDED
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (UDFToDouble(key) < 100.0) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        GlobalTableId: 1
+#### A masked pattern was here ####
+                        NumFilesPerFileSink: 1
+                        Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1
+                              columns.types string:string
+                              serialization.format 1
+                              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        TotalFiles: 1
+                        GatherStats: false
+                        MultiFileSpray: false
+            Path -> Alias:
+#### A masked pattern was here ####
+            Path -> Partition:
+#### A masked pattern was here ####
+                Partition
+                  base file name: src
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                    bucket_count -1
+                    column.name.delimiter ,
+                    columns key,value
+                    columns.comments 'default','default'
+                    columns.types string:string
+#### A masked pattern was here ####
+                    name default.src
+                    numFiles 1
+                    numRows 500
+                    rawDataSize 5312
+                    serialization.ddl struct src { string key, string value}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 5812
+#### 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:
+                      COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                      bucket_count -1
+                      column.name.delimiter ,
+                      columns key,value
+                      columns.comments 'default','default'
+                      columns.types string:string
+#### A masked pattern was here ####
+                      name default.src
+                      numFiles 1
+                      numRows 500
+                      rawDataSize 5312
+                      serialization.ddl struct src { string key, string value}
+                      serialization.format 1
+                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      totalSize 5812
+#### A masked pattern was here ####
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.src
+                  name: default.src
+            Truncated Path -> Alias:
+              /src [null-subquery1:$hdt$_0-subquery1:src]
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (UDFToDouble(key) > 100.0) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        GlobalTableId: 1
+#### A masked pattern was here ####
+                        NumFilesPerFileSink: 1
+                        Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1
+                              columns.types string:string
+                              serialization.format 1
+                              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        TotalFiles: 1
+                        GatherStats: false
+                        MultiFileSpray: false
+            Path -> Alias:
+#### A masked pattern was here ####
+            Path -> Partition:
+#### A masked pattern was here ####
+                Partition
+                  base file name: src
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                    bucket_count -1
+                    column.name.delimiter ,
+                    columns key,value
+                    columns.comments 'default','default'
+                    columns.types string:string
+#### A masked pattern was here ####
+                    name default.src
+                    numFiles 1
+                    numRows 500
+                    rawDataSize 5312
+                    serialization.ddl struct src { string key, string value}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 5812
+#### 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:
+                      COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                      bucket_count -1
+                      column.name.delimiter ,
+                      columns key,value
+                      columns.comments 'default','default'
+                      columns.types string:string
+#### A masked pattern was here ####
+                      name default.src
+                      numFiles 1
+                      numRows 500
+                      rawDataSize 5312
+                      serialization.ddl struct src { string key, string value}
+                      serialization.format 1
+                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      totalSize 5812
+#### A masked pattern was here ####
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.src
+                  name: default.src
+            Truncated Path -> Alias:
+              /src [null-subquery2:$hdt$_0-subquery2:src]
+
+  Stage: Stage-0
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+          destination: target/warehouse/union_merge.out
+
+PREHOOK: query: FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: target/warehouse/union_merge.out
+POSTHOOK: query: FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: target/warehouse/union_merge.out
+Found 2 items
+#### A masked pattern was here ####
+PREHOOK: query: EXPLAIN EXTENDED
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-6 depends on stages: Stage-1 , consists of Stage-3, Stage-2, Stage-4
+  Stage-3
+  Stage-0 depends on stages: Stage-3, Stage-2, Stage-5
+  Stage-2
+  Stage-4
+  Stage-5 depends on stages: Stage-4
+
+STAGE PLANS:
+  Stage: Stage-1
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (UDFToDouble(key) < 100.0) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        GlobalTableId: 1
+#### A masked pattern was here ####
+                        NumFilesPerFileSink: 1
+                        Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1
+                              columns.types string:string
+                              serialization.format 1
+                              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        TotalFiles: 1
+                        GatherStats: false
+                        MultiFileSpray: false
+            Path -> Alias:
+#### A masked pattern was here ####
+            Path -> Partition:
+#### A masked pattern was here ####
+                Partition
+                  base file name: src
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                    bucket_count -1
+                    column.name.delimiter ,
+                    columns key,value
+                    columns.comments 'default','default'
+                    columns.types string:string
+#### A masked pattern was here ####
+                    name default.src
+                    numFiles 1
+                    numRows 500
+                    rawDataSize 5312
+                    serialization.ddl struct src { string key, string value}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 5812
+#### 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:
+                      COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                      bucket_count -1
+                      column.name.delimiter ,
+                      columns key,value
+                      columns.comments 'default','default'
+                      columns.types string:string
+#### A masked pattern was here ####
+                      name default.src
+                      numFiles 1
+                      numRows 500
+                      rawDataSize 5312
+                      serialization.ddl struct src { string key, string value}
+                      serialization.format 1
+                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      totalSize 5812
+#### A masked pattern was here ####
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.src
+                  name: default.src
+            Truncated Path -> Alias:
+              /src [null-subquery1:$hdt$_0-subquery1:src]
+        Map 2 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+                  GatherStats: false
+                  Filter Operator
+                    isSamplingPred: false
+                    predicate: (UDFToDouble(key) > 100.0) (type: boolean)
+                    Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                    Select Operator
+                      expressions: key (type: string), value (type: string)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+                      File Output Operator
+                        compressed: false
+                        GlobalTableId: 1
+#### A masked pattern was here ####
+                        NumFilesPerFileSink: 1
+                        Statistics: Num rows: 332 Data size: 3526 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+                        table:
+                            input format: org.apache.hadoop.mapred.TextInputFormat
+                            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                            properties:
+                              columns _col0,_col1
+                              columns.types string:string
+                              serialization.format 1
+                              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        TotalFiles: 1
+                        GatherStats: false
+                        MultiFileSpray: false
+            Path -> Alias:
+#### A masked pattern was here ####
+            Path -> Partition:
+#### A masked pattern was here ####
+                Partition
+                  base file name: src
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                  properties:
+                    COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                    bucket_count -1
+                    column.name.delimiter ,
+                    columns key,value
+                    columns.comments 'default','default'
+                    columns.types string:string
+#### A masked pattern was here ####
+                    name default.src
+                    numFiles 1
+                    numRows 500
+                    rawDataSize 5312
+                    serialization.ddl struct src { string key, string value}
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    totalSize 5812
+#### 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:
+                      COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+                      bucket_count -1
+                      column.name.delimiter ,
+                      columns key,value
+                      columns.comments 'default','default'
+                      columns.types string:string
+#### A masked pattern was here ####
+                      name default.src
+                      numFiles 1
+                      numRows 500
+                      rawDataSize 5312
+                      serialization.ddl struct src { string key, string value}
+                      serialization.format 1
+                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      totalSize 5812
+#### A masked pattern was here ####
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    name: default.src
+                  name: default.src
+            Truncated Path -> Alias:
+              /src [null-subquery2:$hdt$_0-subquery2:src]
+
+  Stage: Stage-6
+    Conditional Operator
+
+  Stage: Stage-3
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+  Stage: Stage-0
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+          destination: target/warehouse/union_merge.out
+
+  Stage: Stage-2
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Spark Merge File Work 
+            Map Operator Tree:
+                TableScan
+                  GatherStats: false
+                  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:
+                          columns _col0,_col1
+                          columns.types string:string
+                          serialization.format 1
+                          serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    TotalFiles: 1
+                    GatherStats: false
+                    MultiFileSpray: 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:
+                    columns _col0,_col1
+                    columns.types string:string
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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:
+                      columns _col0,_col1
+                      columns.types string:string
+                      serialization.format 1
+                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Truncated Path -> Alias:
+#### A masked pattern was here ####
+
+  Stage: Stage-4
+    Spark
+#### A masked pattern was here ####
+      Vertices:
+        Spark Merge File Work 
+            Map Operator Tree:
+                TableScan
+                  GatherStats: false
+                  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:
+                          columns _col0,_col1
+                          columns.types string:string
+                          serialization.format 1
+                          serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    TotalFiles: 1
+                    GatherStats: false
+                    MultiFileSpray: 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:
+                    columns _col0,_col1
+                    columns.types string:string
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                  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:
+                      columns _col0,_col1
+                      columns.types string:string
+                      serialization.format 1
+                      serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                    serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+            Truncated Path -> Alias:
+#### A masked pattern was here ####
+
+  Stage: Stage-5
+    Move Operator
+      files:
+          hdfs directory: true
+#### A masked pattern was here ####
+
+PREHOOK: query: FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: target/warehouse/union_merge.out
+POSTHOOK: query: FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union_merge.out' SELECT unioninput.*
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: target/warehouse/union_merge.out
+Found 1 items
+#### A masked pattern was here ####

http://git-wip-us.apache.org/repos/asf/hive/blob/318822b1/testutils/ptest2/conf/deployed/master-mr2.properties
----------------------------------------------------------------------
diff --git a/testutils/ptest2/conf/deployed/master-mr2.properties b/testutils/ptest2/conf/deployed/master-mr2.properties
index df2ee11..7edc307 100644
--- a/testutils/ptest2/conf/deployed/master-mr2.properties
+++ b/testutils/ptest2/conf/deployed/master-mr2.properties
@@ -148,14 +148,14 @@ qFileTest.spark.directory = ql/src/test/queries/clientpositive
 qFileTest.spark.batchSize = 15
 qFileTest.spark.queryFilesProperty = qfile
 qFileTest.spark.include = normal
-qFileTest.spark.groups.normal = mainProperties.${spark.query.files}
+qFileTest.spark.groups.normal = mainProperties.${spark.query.files} mainProperties.${spark.only.query.files}
 
 qFileTest.miniSparkOnYarn.driver = TestMiniSparkOnYarnCliDriver
 qFileTest.miniSparkOnYarn.directory = ql/src/test/queries/clientpositive
 qFileTest.miniSparkOnYarn.batchSize = 20
 qFileTest.miniSparkOnYarn.queryFilesProperty = qfile
 qFileTest.miniSparkOnYarn.include = normal
-qFileTest.miniSparkOnYarn.groups.normal = mainProperties.${miniSparkOnYarn.query.files} mainProperties.${spark.only.query.files}
+qFileTest.miniSparkOnYarn.groups.normal = mainProperties.${miniSparkOnYarn.query.files} mainProperties.${miniSparkOnYarn.only.query.files}
 
 qFileTest.miniLlap.driver = TestMiniLlapCliDriver
 qFileTest.miniLlap.directory = ql/src/test/queries/clientpositive