You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ke...@apache.org on 2013/04/05 20:34:19 UTC

svn commit: r1465085 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/exec/ java/org/apache/hadoop/hive/ql/profiler/ test/queries/clientpositive/ test/results/clientpositive/

Author: kevinwilfong
Date: Fri Apr  5 18:34:18 2013
New Revision: 1465085

URL: http://svn.apache.org/r1465085
Log:
HIVE-4151. HiveProfiler NPE with ScriptOperator. (Pamela Vagata via kevinwilfong)

Added:
    hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_script0.q
    hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_union0.q
    hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_script0.q.out
    hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_union0.q.out
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorHookContext.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilePublisher.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfiler.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStats.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStatsAggregator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerUtils.java
    hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler0.q
    hive/trunk/ql/src/test/results/clientpositive/hiveprofiler0.q.out

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java Fri Apr  5 18:34:18 2013
@@ -525,7 +525,7 @@ public abstract class Operator<T extends
     if (fatalError) {
       return;
     }
-    OperatorHookContext opHookContext = new OperatorHookContext(this, row);
+    OperatorHookContext opHookContext = new OperatorHookContext(this, row, tag);
     preProcessCounter();
     enterOperatorHooks(opHookContext);
     processOp(row, tag);
@@ -612,7 +612,7 @@ public abstract class Operator<T extends
 
     LOG.info(id + " forwarded " + cntr + " rows");
 
-    closeOperatorHooks(new OperatorHookContext(this, null));
+    closeOperatorHooks(new OperatorHookContext(this));
     // call the operator specific close routine
     closeOp(abort);
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorHookContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorHookContext.java?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorHookContext.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorHookContext.java Fri Apr  5 18:34:18 2013
@@ -18,26 +18,44 @@
 
 package org.apache.hadoop.hive.ql.exec;
 
+import java.util.List;
+
 public class OperatorHookContext {
-  private String operatorName;
-  private String operatorId;
-  private Object currentRow;
+  private final String operatorName;
+  private final String operatorId;
+  private final Object currentRow;
+  private final int parentTag;
   private Operator operator;
-  public OperatorHookContext(Operator op, Object row) {
-    this(op.getName(), op.getIdentifier(), row);
+
+  public OperatorHookContext(Operator op) {
+    this(op, null, -1);
+  }
+
+  public OperatorHookContext(Operator op, Object row, int tag) {
+    this(op.getName(), op.getIdentifier(), row, tag);
     this.operator = op;
   }
 
-  private OperatorHookContext(String opName, String opId, Object row) {
+  private OperatorHookContext(String opName, String opId, Object row, int tag) {
     operatorName = opName;
     operatorId = opId;
     currentRow = row;
+    parentTag = tag;
   }
 
   public Operator getOperator() {
     return operator;
   }
 
+  public Operator getParentOperator() {
+    List<Operator> parents = this.operator.getParentOperators();
+    if (parents == null || parents.isEmpty()) {
+      return null;
+    }
+    return (Operator)(this.operator.getParentOperators().get(this.parentTag));
+
+  }
+
   public String getOperatorName() {
     return operatorName;
   }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilePublisher.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilePublisher.java?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilePublisher.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilePublisher.java Fri Apr  5 18:34:18 2013
@@ -45,7 +45,7 @@ public class HiveProfilePublisher {
       }
       info.getConnection().close();
       return true;
-    } catch (SQLException e) {
+    } catch (Exception e) {
       LOG.error("Error during JDBC termination. ", e);
       return false;
     }
@@ -100,7 +100,7 @@ public class HiveProfilePublisher {
         };
         PreparedStatement insStmt = info.getInsert(stats);
         Utilities.executeWithRetry(execUpdate, insStmt, info.getWaitWindow(), info.getMaxRetries());
-      } catch (SQLException e) {
+      } catch (Exception e) {
         LOG.error("ERROR during publishing profiling data. ", e);
         return false;
       }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfiler.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfiler.java?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfiler.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfiler.java Fri Apr  5 18:34:18 2013
@@ -17,18 +17,14 @@
  */
 package org.apache.hadoop.hive.ql.profiler;
 
-import java.lang.System;
-import java.util.LinkedList;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Iterator;
 import java.util.Collection;
-import java.util.List;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.ql.exec.OperatorHook;
 import org.apache.hadoop.hive.ql.exec.OperatorHookContext;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -37,41 +33,40 @@ public class HiveProfiler implements Ope
   private final Log LOG = LogFactory.getLog(this.getClass().getName());
   private static final HiveProfilePublisher pub = new HiveProfilePublisher();
 
-  private LinkedList<HiveProfilerEntry> operatorCallStack =
-    new LinkedList<HiveProfilerEntry>();
+  private final Map<String, HiveProfilerEntry> operatorCallStack =
+    new ConcurrentHashMap<String, HiveProfilerEntry>();
 
   // Aggregates stats for each operator in memory so that stats are written to DB
   // all at once - this allows the profiler to be extremely lightweight in
   // communication with the DB
-  private Map<String, HiveProfilerStats> aggrStats =
-    new HashMap<String, HiveProfilerStats>();
+  private final Map<String, HiveProfilerStats> aggrStats =
+    new ConcurrentHashMap<String, HiveProfilerStats>();
 
   public void enter(OperatorHookContext opHookContext) throws HiveException {
+    String opLevelAnnoName = HiveProfilerUtils.getLevelAnnotatedName(opHookContext);
     HiveProfilerEntry curEntry = new HiveProfilerEntry(opHookContext);
-    operatorCallStack.addFirst(curEntry);
+    assert(operatorCallStack.get(opLevelAnnoName) == null);
+    operatorCallStack.put(opLevelAnnoName, curEntry);
   }
 
-  private void exit(HiveProfilerEntry curEntry, HiveProfilerEntry parentEntry) {
+  private void exit(HiveProfilerEntry curEntry) {
     OperatorHookContext opHookContext = curEntry.getOperatorHookContext();
-
     // update the metrics we are
     long exitTime = System.nanoTime();
     long wallTime = exitTime - curEntry.wallStartTime;
 
     String opName = opHookContext.getOperatorName();
 
-    OperatorHookContext parentContext =
-      parentEntry != null ? parentEntry.getOperatorHookContext() :
-        null;
     Configuration conf = opHookContext.getOperator().getConfiguration();
 
-    String opId = opHookContext.getOperatorId();
-    if (aggrStats.containsKey(opId)) {
-      aggrStats.get(opId).updateStats(wallTime, 1);
+    String opLevelAnnoName = HiveProfilerUtils.getLevelAnnotatedName(opHookContext);
+
+    if (aggrStats.containsKey(opLevelAnnoName)) {
+      aggrStats.get(opLevelAnnoName).updateStats(wallTime, 1);
     } else {
       HiveProfilerStats stats =
-        new HiveProfilerStats(opHookContext, parentContext, 1, wallTime, conf);
-      aggrStats.put(opId, stats);
+        new HiveProfilerStats(opHookContext, 1, wallTime, conf);
+      aggrStats.put(opLevelAnnoName, stats);
     }
 
   }
@@ -79,16 +74,17 @@ public class HiveProfiler implements Ope
     if (operatorCallStack.isEmpty()) {
       LOG.error("Unexpected state: Operator Call Stack is empty on exit.");
     }
+    String opLevelAnnoName = HiveProfilerUtils.getLevelAnnotatedName(opHookContext);
+
+    HiveProfilerEntry curEntry = operatorCallStack.get(opLevelAnnoName);
 
-    // grab the top item on the call stack since that should be
-    // the first operator to exit.
-    HiveProfilerEntry curEntry = operatorCallStack.poll();
     if (!curEntry.getOperatorHookContext().equals(opHookContext)) {
       LOG.error("Expected to exit from: " + curEntry.getOperatorHookContext().toString() +
         " but exit called on " + opHookContext.toString());
     }
-    HiveProfilerEntry parentEntry = operatorCallStack.peekFirst();
-    exit(curEntry, parentEntry);
+
+    exit(curEntry);
+    operatorCallStack.remove(opLevelAnnoName);
   }
 
   public void close(OperatorHookContext opHookContext) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStats.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStats.java?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStats.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStats.java Fri Apr  5 18:34:18 2013
@@ -21,6 +21,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.OperatorHookContext;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 
@@ -50,7 +51,7 @@ public class HiveProfilerStats {
     Columns.CALL_COUNT
   };
 
-  private Map<String, String> stats = new HashMap<String, String>();
+  private final Map<String, String> stats = new HashMap<String, String>();
 
   long callCount;
   long inclTime;
@@ -58,16 +59,14 @@ public class HiveProfilerStats {
 
   protected HiveProfilerStats(
     OperatorHookContext opHookContext,
-    OperatorHookContext parentOpHookContext,
     long callCount, long wallTime, Configuration conf) {
     this.callCount = callCount;
     this.inclTime = wallTime;
     this.taskId = Utilities.getTaskId(conf);
-    populateStatsMap(opHookContext, parentOpHookContext, conf);
+    populateStatsMap(opHookContext, conf);
   }
 
   private void populateStatsMap(OperatorHookContext opHookContext,
-    OperatorHookContext parentOpHookContext,
     Configuration conf) {
     String queryId =
       conf == null ? "no conf" : HiveConf.getVar(conf, HiveConf.ConfVars.HIVEQUERYID);
@@ -78,17 +77,14 @@ public class HiveProfilerStats {
     stats.put(
       Columns.OPERATOR_ID, opHookContext.getOperatorId());
 
-    String parentOpName = parentOpHookContext == null ? "" : parentOpHookContext.getOperatorName();
+    Operator parent = opHookContext.getParentOperator();
+    String parentOpName = parent == null ? "" : parent.getName();
     stats.put(Columns.PARENT_OPERATOR_NAME, parentOpName);
 
-
-    String parentOpId = parentOpHookContext == null ? "-1" : parentOpHookContext.getOperatorId();
+    String parentOpId = parent == null ? "-1" : parent.getIdentifier();
     stats.put(Columns.PARENT_OPERATOR_ID, parentOpId);
 
-    String levelAnnoOpName = opName + "_" + opHookContext.getOperatorId();
-    String levelAnnoName = parentOpHookContext == null ? "main() ==> " + levelAnnoOpName :
-      parentOpName + "_" +  parentOpId + " ==> " + levelAnnoOpName;
-    stats.put(Columns.LEVEL_ANNO_NAME, levelAnnoName);
+    stats.put(Columns.LEVEL_ANNO_NAME, HiveProfilerUtils.getLevelAnnotatedName(opHookContext));
 
   }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStatsAggregator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStatsAggregator.java?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStatsAggregator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerStatsAggregator.java Fri Apr  5 18:34:18 2013
@@ -110,7 +110,7 @@ public class HiveProfilerStatsAggregator
           stats.put(levelAnnoName, curStat);
         }
       }
-    } catch (SQLException e) {
+    } catch (Exception e) {
       LOG.error("Error Aggregating Stats", e);
     }
   }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerUtils.java?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/profiler/HiveProfilerUtils.java Fri Apr  5 18:34:18 2013
@@ -18,14 +18,15 @@
 package org.apache.hadoop.hive.ql.profiler;
 
 import java.sql.Connection;
-import java.sql.Statement;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.MapOperator;
+import org.apache.hadoop.hive.ql.exec.OperatorHookContext;
 
 public class HiveProfilerUtils {
   public static void createTableIfNonExistent(HiveProfilerConnectionInfo info,
@@ -57,4 +58,16 @@ public class HiveProfilerUtils {
     }
     return true;
   }
+
+  public static String getLevelAnnotatedName(OperatorHookContext opHookContext) {
+    Operator parent = opHookContext.getParentOperator();
+    if (parent != null && parent instanceof MapOperator) {
+      parent = null;
+    }
+    Operator op = opHookContext.getOperator();
+    String parentOpName = parent == null ? "" : parent.getName();
+    String parentOpId = parent == null ? "main()" : parent.getOperatorId();
+    String levelAnnoName = parentOpId + " ==> " + op.getOperatorId();
+    return levelAnnoName;
+  }
 }

Modified: hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler0.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler0.q?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler0.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler0.q Fri Apr  5 18:34:18 2013
@@ -1,6 +1,6 @@
 set hive.exec.operator.hooks=org.apache.hadoop.hive.ql.profiler.HiveProfiler;
 set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.HiveProfilerResultsHook;
-SET hive.exec.mode.local.auto=false;
 SET hive.task.progress=true;
 
 select count(1) from src;
+explain select count(1) from src;

Added: hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_script0.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_script0.q?rev=1465085&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_script0.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_script0.q Fri Apr  5 18:34:18 2013
@@ -0,0 +1,15 @@
+SET hive.task.progress=true;
+set hive.exec.operator.hooks=org.apache.hadoop.hive.ql.profiler.HiveProfiler;
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.HiveProfilerResultsHook;
+ADD FILE src/test/scripts/testgrep;
+
+
+-- checking that script operator does not cause NPE
+-- Derby strangeness is causing the output collector for the Hive Profiler to not get output during DB read 
+
+SELECT TRANSFORM(src.key, src.value)
+       USING 'testgrep' AS (tkey, tvalue)
+FROM src
+
+
+

Added: hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_union0.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_union0.q?rev=1465085&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_union0.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/hiveprofiler_union0.q Fri Apr  5 18:34:18 2013
@@ -0,0 +1,19 @@
+set hive.exec.operator.hooks=org.apache.hadoop.hive.ql.profiler.HiveProfiler;
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.HiveProfilerResultsHook;
+SET hive.task.progress=true;
+FROM (
+  FROM src select src.key, src.value WHERE src.key < 100
+  UNION ALL
+  FROM src SELECT src.* WHERE src.key > 100
+) unioninput
+SELECT unioninput.*;
+
+explain
+  FROM (
+    FROM src select src.key, src.value WHERE src.key < 100
+    UNION ALL
+    FROM src SELECT src.* WHERE src.key > 100
+  ) unioninput
+  SELECT unioninput.*;
+
+

Modified: hive/trunk/ql/src/test/results/clientpositive/hiveprofiler0.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/hiveprofiler0.q.out?rev=1465085&r1=1465084&r2=1465085&view=diff
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/hiveprofiler0.q.out (original)
+++ hive/trunk/ql/src/test/results/clientpositive/hiveprofiler0.q.out Fri Apr  5 18:34:18 2013
@@ -2,11 +2,64 @@ PREHOOK: query: select count(1) from src
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
+GBY_2 ==> RS_3: 1
 TS_0 ==> SEL_1: 500
-main() ==> RS_3: 1
 SEL_1 ==> GBY_2: 500
-main() ==> SEL_5: 1
+GBY_4 ==> SEL_5: 1
 main() ==> GBY_4: 1
 main() ==> TS_0: 500
 SEL_5 ==> FS_6: 1
 500
+PREHOOK: query: explain select count(1) from src
+PREHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_FUNCTION count 1)))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        src 
+          TableScan
+            alias: src
+            Select Operator
+              Group By Operator
+                aggregations:
+                      expr: count(1)
+                bucketGroup: false
+                mode: hash
+                outputColumnNames: _col0
+                Reduce Output Operator
+                  sort order: 
+                  tag: -1
+                  value expressions:
+                        expr: _col0
+                        type: bigint
+      Reduce Operator Tree:
+        Group By Operator
+          aggregations:
+                expr: count(VALUE._col0)
+          bucketGroup: false
+          mode: mergepartial
+          outputColumnNames: _col0
+          Select Operator
+            expressions:
+                  expr: _col0
+                  type: bigint
+            outputColumnNames: _col0
+            File Output Operator
+              compressed: false
+              GlobalTableId: 0
+              table:
+                  input format: org.apache.hadoop.mapred.TextInputFormat
+                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+

Added: hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_script0.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_script0.q.out?rev=1465085&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_script0.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_script0.q.out Fri Apr  5 18:34:18 2013
@@ -0,0 +1,18 @@
+PREHOOK: query: -- checking that script operator does not cause NPE
+-- Derby strangeness is causing the output collector for the Hive Profiler to not get output during DB read 
+
+SELECT TRANSFORM(src.key, src.value)
+       USING 'testgrep' AS (tkey, tvalue)
+FROM src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+103	val_103
+100	val_100
+10	val_10
+103	val_103
+104	val_104
+105	val_105
+104	val_104
+310	val_310
+100	val_100

Added: hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_union0.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_union0.q.out?rev=1465085&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_union0.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/hiveprofiler_union0.q.out Fri Apr  5 18:34:18 2013
@@ -0,0 +1,598 @@
+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
+SELECT unioninput.*
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+#### A masked pattern was here ####
+SEL_5 ==> UNION_6: 414
+FIL_10 ==> SEL_5: 414
+TS_0 ==> FIL_9: 500
+TS_3 ==> FIL_10: 500
+UNION_6 ==> SEL_7: 498
+SEL_2 ==> UNION_6: 84
+FIL_9 ==> SEL_2: 84
+main() ==> TS_3: 500
+main() ==> TS_0: 500
+SEL_7 ==> FS_8: 498
+238	val_238
+86	val_86
+311	val_311
+27	val_27
+165	val_165
+409	val_409
+255	val_255
+278	val_278
+98	val_98
+484	val_484
+265	val_265
+193	val_193
+401	val_401
+150	val_150
+273	val_273
+224	val_224
+369	val_369
+66	val_66
+128	val_128
+213	val_213
+146	val_146
+406	val_406
+429	val_429
+374	val_374
+152	val_152
+469	val_469
+145	val_145
+495	val_495
+37	val_37
+327	val_327
+281	val_281
+277	val_277
+209	val_209
+15	val_15
+82	val_82
+403	val_403
+166	val_166
+417	val_417
+430	val_430
+252	val_252
+292	val_292
+219	val_219
+287	val_287
+153	val_153
+193	val_193
+338	val_338
+446	val_446
+459	val_459
+394	val_394
+237	val_237
+482	val_482
+174	val_174
+413	val_413
+494	val_494
+207	val_207
+199	val_199
+466	val_466
+208	val_208
+174	val_174
+399	val_399
+396	val_396
+247	val_247
+417	val_417
+489	val_489
+162	val_162
+377	val_377
+397	val_397
+309	val_309
+365	val_365
+266	val_266
+439	val_439
+342	val_342
+367	val_367
+325	val_325
+167	val_167
+195	val_195
+475	val_475
+17	val_17
+113	val_113
+155	val_155
+203	val_203
+339	val_339
+0	val_0
+455	val_455
+128	val_128
+311	val_311
+316	val_316
+57	val_57
+302	val_302
+205	val_205
+149	val_149
+438	val_438
+345	val_345
+129	val_129
+170	val_170
+20	val_20
+489	val_489
+157	val_157
+378	val_378
+221	val_221
+92	val_92
+111	val_111
+47	val_47
+72	val_72
+4	val_4
+280	val_280
+35	val_35
+427	val_427
+277	val_277
+208	val_208
+356	val_356
+399	val_399
+169	val_169
+382	val_382
+498	val_498
+125	val_125
+386	val_386
+437	val_437
+469	val_469
+192	val_192
+286	val_286
+187	val_187
+176	val_176
+54	val_54
+459	val_459
+51	val_51
+138	val_138
+103	val_103
+239	val_239
+213	val_213
+216	val_216
+430	val_430
+278	val_278
+176	val_176
+289	val_289
+221	val_221
+65	val_65
+318	val_318
+332	val_332
+311	val_311
+275	val_275
+137	val_137
+241	val_241
+83	val_83
+333	val_333
+180	val_180
+284	val_284
+12	val_12
+230	val_230
+181	val_181
+67	val_67
+260	val_260
+404	val_404
+384	val_384
+489	val_489
+353	val_353
+373	val_373
+272	val_272
+138	val_138
+217	val_217
+84	val_84
+348	val_348
+466	val_466
+58	val_58
+8	val_8
+411	val_411
+230	val_230
+208	val_208
+348	val_348
+24	val_24
+463	val_463
+431	val_431
+179	val_179
+172	val_172
+42	val_42
+129	val_129
+158	val_158
+119	val_119
+496	val_496
+0	val_0
+322	val_322
+197	val_197
+468	val_468
+393	val_393
+454	val_454
+298	val_298
+199	val_199
+191	val_191
+418	val_418
+96	val_96
+26	val_26
+165	val_165
+327	val_327
+230	val_230
+205	val_205
+120	val_120
+131	val_131
+51	val_51
+404	val_404
+43	val_43
+436	val_436
+156	val_156
+469	val_469
+468	val_468
+308	val_308
+95	val_95
+196	val_196
+288	val_288
+481	val_481
+457	val_457
+98	val_98
+282	val_282
+197	val_197
+187	val_187
+318	val_318
+318	val_318
+409	val_409
+470	val_470
+137	val_137
+369	val_369
+316	val_316
+169	val_169
+413	val_413
+85	val_85
+77	val_77
+0	val_0
+490	val_490
+87	val_87
+364	val_364
+179	val_179
+118	val_118
+134	val_134
+395	val_395
+282	val_282
+138	val_138
+238	val_238
+419	val_419
+15	val_15
+118	val_118
+72	val_72
+90	val_90
+307	val_307
+19	val_19
+435	val_435
+10	val_10
+277	val_277
+273	val_273
+306	val_306
+224	val_224
+309	val_309
+389	val_389
+327	val_327
+242	val_242
+369	val_369
+392	val_392
+272	val_272
+331	val_331
+401	val_401
+242	val_242
+452	val_452
+177	val_177
+226	val_226
+5	val_5
+497	val_497
+402	val_402
+396	val_396
+317	val_317
+395	val_395
+58	val_58
+35	val_35
+336	val_336
+95	val_95
+11	val_11
+168	val_168
+34	val_34
+229	val_229
+233	val_233
+143	val_143
+472	val_472
+322	val_322
+498	val_498
+160	val_160
+195	val_195
+42	val_42
+321	val_321
+430	val_430
+119	val_119
+489	val_489
+458	val_458
+78	val_78
+76	val_76
+41	val_41
+223	val_223
+492	val_492
+149	val_149
+449	val_449
+218	val_218
+228	val_228
+138	val_138
+453	val_453
+30	val_30
+209	val_209
+64	val_64
+468	val_468
+76	val_76
+74	val_74
+342	val_342
+69	val_69
+230	val_230
+33	val_33
+368	val_368
+103	val_103
+296	val_296
+113	val_113
+216	val_216
+367	val_367
+344	val_344
+167	val_167
+274	val_274
+219	val_219
+239	val_239
+485	val_485
+116	val_116
+223	val_223
+256	val_256
+263	val_263
+70	val_70
+487	val_487
+480	val_480
+401	val_401
+288	val_288
+191	val_191
+5	val_5
+244	val_244
+438	val_438
+128	val_128
+467	val_467
+432	val_432
+202	val_202
+316	val_316
+229	val_229
+469	val_469
+463	val_463
+280	val_280
+2	val_2
+35	val_35
+283	val_283
+331	val_331
+235	val_235
+80	val_80
+44	val_44
+193	val_193
+321	val_321
+335	val_335
+104	val_104
+466	val_466
+366	val_366
+175	val_175
+403	val_403
+483	val_483
+53	val_53
+105	val_105
+257	val_257
+406	val_406
+409	val_409
+190	val_190
+406	val_406
+401	val_401
+114	val_114
+258	val_258
+90	val_90
+203	val_203
+262	val_262
+348	val_348
+424	val_424
+12	val_12
+396	val_396
+201	val_201
+217	val_217
+164	val_164
+431	val_431
+454	val_454
+478	val_478
+298	val_298
+125	val_125
+431	val_431
+164	val_164
+424	val_424
+187	val_187
+382	val_382
+5	val_5
+70	val_70
+397	val_397
+480	val_480
+291	val_291
+24	val_24
+351	val_351
+255	val_255
+104	val_104
+70	val_70
+163	val_163
+438	val_438
+119	val_119
+414	val_414
+200	val_200
+491	val_491
+237	val_237
+439	val_439
+360	val_360
+248	val_248
+479	val_479
+305	val_305
+417	val_417
+199	val_199
+444	val_444
+120	val_120
+429	val_429
+169	val_169
+443	val_443
+323	val_323
+325	val_325
+277	val_277
+230	val_230
+478	val_478
+178	val_178
+468	val_468
+310	val_310
+317	val_317
+333	val_333
+493	val_493
+460	val_460
+207	val_207
+249	val_249
+265	val_265
+480	val_480
+83	val_83
+136	val_136
+353	val_353
+172	val_172
+214	val_214
+462	val_462
+233	val_233
+406	val_406
+133	val_133
+175	val_175
+189	val_189
+454	val_454
+375	val_375
+401	val_401
+421	val_421
+407	val_407
+384	val_384
+256	val_256
+26	val_26
+134	val_134
+67	val_67
+384	val_384
+379	val_379
+18	val_18
+462	val_462
+492	val_492
+298	val_298
+9	val_9
+341	val_341
+498	val_498
+146	val_146
+458	val_458
+362	val_362
+186	val_186
+285	val_285
+348	val_348
+167	val_167
+18	val_18
+273	val_273
+183	val_183
+281	val_281
+344	val_344
+97	val_97
+469	val_469
+315	val_315
+84	val_84
+28	val_28
+37	val_37
+448	val_448
+152	val_152
+348	val_348
+307	val_307
+194	val_194
+414	val_414
+477	val_477
+222	val_222
+126	val_126
+90	val_90
+169	val_169
+403	val_403
+400	val_400
+200	val_200
+97	val_97
+PREHOOK: query: explain
+  FROM (
+    FROM src select src.key, src.value WHERE src.key < 100
+    UNION ALL
+    FROM src SELECT src.* WHERE src.key > 100
+  ) unioninput
+  SELECT unioninput.*
+PREHOOK: type: QUERY
+ABSTRACT SYNTAX TREE:
+  (TOK_QUERY (TOK_FROM (TOK_SUBQUERY (TOK_UNION (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) key)) (TOK_SELEXPR (. (TOK_TABLE_OR_COL src) value))) (TOK_WHERE (< (. (TOK_TABLE_OR_COL src) key) 100)))) (TOK_QUERY (TOK_FROM (TOK_TABREF (TOK_TABNAME src))) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME src)))) (TOK_WHERE (> (. (TOK_TABLE_OR_COL src) key) 100))))) unioninput)) (TOK_INSERT (TOK_DESTINATION (TOK_DIR TOK_TMP_FILE)) (TOK_SELECT (TOK_SELEXPR (TOK_ALLCOLREF (TOK_TABNAME unioninput))))))
+
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-0 is a root stage
+
+STAGE PLANS:
+  Stage: Stage-1
+    Map Reduce
+      Alias -> Map Operator Tree:
+        null-subquery1:unioninput-subquery1:src 
+          TableScan
+            alias: src
+            Filter Operator
+              predicate:
+                  expr: (key < 100.0)
+                  type: boolean
+              Select Operator
+                expressions:
+                      expr: key
+                      type: string
+                      expr: value
+                      type: string
+                outputColumnNames: _col0, _col1
+                Union
+                  Select Operator
+                    expressions:
+                          expr: _col0
+                          type: string
+                          expr: _col1
+                          type: string
+                    outputColumnNames: _col0, _col1
+                    File Output Operator
+                      compressed: false
+                      GlobalTableId: 0
+                      table:
+                          input format: org.apache.hadoop.mapred.TextInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+        null-subquery2:unioninput-subquery2:src 
+          TableScan
+            alias: src
+            Filter Operator
+              predicate:
+                  expr: (key > 100.0)
+                  type: boolean
+              Select Operator
+                expressions:
+                      expr: key
+                      type: string
+                      expr: value
+                      type: string
+                outputColumnNames: _col0, _col1
+                Union
+                  Select Operator
+                    expressions:
+                          expr: _col0
+                          type: string
+                          expr: _col1
+                          type: string
+                    outputColumnNames: _col0, _col1
+                    File Output Operator
+                      compressed: false
+                      GlobalTableId: 0
+                      table:
+                          input format: org.apache.hadoop.mapred.TextInputFormat
+                          output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+
+  Stage: Stage-0
+    Fetch Operator
+      limit: -1
+
+