You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ek...@apache.org on 2018/05/01 00:53:54 UTC

hive git commit: HIVE-19322 - broken test: TestNegativeMinimrCliDriver#testCliDriver[minimr_broken_pipe] (Eugene Koifman, reviewed by Jason Dere)

Repository: hive
Updated Branches:
  refs/heads/master 3c4a9c6bc -> ee6a53268


HIVE-19322 - broken test: TestNegativeMinimrCliDriver#testCliDriver[minimr_broken_pipe] (Eugene Koifman, reviewed by Jason Dere)


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

Branch: refs/heads/master
Commit: ee6a53268989a823bb4a40638d9c93c52658f76c
Parents: 3c4a9c6
Author: Eugene Koifman <ek...@apache.org>
Authored: Mon Apr 30 17:53:45 2018 -0700
Committer: Eugene Koifman <ek...@apache.org>
Committed: Mon Apr 30 17:53:45 2018 -0700

----------------------------------------------------------------------
 .../ql/cache/results/QueryResultsCache.java     |  2 +-
 .../ql/metadata/SessionHiveMetaStoreClient.java | 66 +++++++++++++-------
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |  3 +-
 .../ql/parse/repl/dump/PartitionExport.java     |  9 ++-
 .../hadoop/hive/ql/session/SessionState.java    | 14 +++--
 .../avro_non_nullable_union.q.out               |  1 -
 .../clientnegative/cachingprintstream.q.out     |  2 -
 .../check_constraint_tbl_level.q.out            |  1 -
 .../check_constraint_violation.q.out            |  1 -
 .../clientnegative/compute_stats_long.q.out     |  1 -
 .../test/results/clientnegative/dyn_part3.q.out |  1 -
 .../clientnegative/dyn_part_max_per_node.q.out  |  1 -
 .../dynamic_partitions_with_whitelist.q.out     |  1 -
 .../results/clientnegative/insertsel_fail.q.out |  2 +-
 .../local_mapred_error_cache.q.out              |  1 -
 .../merge_constraint_notnull.q.out              |  1 -
 .../clientnegative/script_broken_pipe2.q.out    |  1 -
 .../clientnegative/script_broken_pipe3.q.out    |  1 -
 .../results/clientnegative/script_error.q.out   |  1 -
 .../results/clientnegative/serde_regex2.q.out   |  1 -
 .../stats_aggregator_error_2.q.out              |  1 -
 .../stats_publisher_error_1.q.out               |  1 -
 .../stats_publisher_error_2.q.out               |  1 -
 .../clientnegative/subquery_corr_in_agg.q.out   |  1 -
 .../subquery_in_implicit_gby.q.out              |  1 -
 .../subquery_notin_implicit_gby.q.out           |  1 -
 .../subquery_scalar_corr_multi_rows.q.out       |  1 -
 .../subquery_scalar_multi_rows.q.out            |  1 -
 .../clientnegative/udf_assert_true.q.out        |  1 -
 .../clientnegative/udf_assert_true2.q.out       |  1 -
 .../clientnegative/udf_reflect_neg.q.out        |  1 -
 .../results/clientnegative/udf_test_error.q.out |  1 -
 .../clientnegative/udf_test_error_reduce.q.out  |  1 -
 33 files changed, 64 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java b/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
index 56a9faa..6734ac5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/cache/results/QueryResultsCache.java
@@ -687,7 +687,7 @@ public final class QueryResultsCache {
       if (readEntity.getType() == Type.TABLE) {
         Table tableUsed = readEntity.getTable();
         Map<String, Table> tempTables =
-            SessionHiveMetaStoreClient.getTempTablesForDatabase(tableUsed.getDbName());
+            SessionHiveMetaStoreClient.getTempTablesForDatabase(tableUsed.getDbName(), tableUsed.getTableName());
         if (tempTables != null && tempTables.containsKey(tableUsed.getTableName())) {
           LOG.info("{} resolves to a temporary table in the current session. This query cannot use the cache.",
               tableUsed.getTableName());

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
index d89df48..071756d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/SessionHiveMetaStoreClient.java
@@ -68,10 +68,22 @@ import org.apache.hadoop.hive.shims.HadoopShims;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.ql.stats.StatsUtils;
 import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
 
+/**
+ * todo: This need review re: thread safety.  Various places (see callsers of
+ * {@link SessionState#setCurrentSessionState(SessionState)}) pass SessionState to forked threads.
+ * Currently it looks like those threads only read metadata but this is fragile.
+ * Also, maps (in SessionState) where tempt table metadata is stored are concurrent and so
+ * any put/get crosses a memory barrier and so does using most {@code java.util.concurrent.*}
+ * so the readers of the objects in these maps should have the most recent view of the object.
+ * But again, could be fragile.
+ */
 public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements IMetaStoreClient {
+  private static final Logger LOG = LoggerFactory.getLogger(SessionHiveMetaStoreClient.class);
 
   SessionHiveMetaStoreClient(Configuration conf, Boolean allowEmbedded) throws MetaException {
     super(conf, null, allowEmbedded);
@@ -174,7 +186,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     List<String> tableNames = super.getAllTables(dbName);
 
     // May need to merge with list of temp tables
-    Map<String, Table> tables = getTempTablesForDatabase(dbName);
+    Map<String, Table> tables = getTempTablesForDatabase(dbName, "?");
     if (tables == null || tables.size() == 0) {
       return tableNames;
     }
@@ -198,7 +210,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     // May need to merge with list of temp tables
     dbName = dbName.toLowerCase();
     tablePattern = tablePattern.toLowerCase();
-    Map<String, Table> tables = getTempTablesForDatabase(dbName);
+    Map<String, Table> tables = getTempTablesForDatabase(dbName, tablePattern);
     if (tables == null || tables.size() == 0) {
       return tableNames;
     }
@@ -224,7 +236,8 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
   public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
       throws MetaException {
     List<TableMeta> tableMetas = super.getTableMeta(dbPatterns, tablePatterns, tableTypes);
-    Map<String, Map<String, Table>> tmpTables = getTempTables();
+    Map<String, Map<String, Table>> tmpTables = getTempTables("dbPatterns='" + dbPatterns +
+        "' tablePatterns='" + tablePatterns + "'");
     if (tmpTables.isEmpty()) {
       return tableMetas;
     }
@@ -426,7 +439,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
 
     SessionState ss = SessionState.get();
     if (ss == null) {
-      throw new MetaException("No current SessionState, cannot create temporary table"
+      throw new MetaException("No current SessionState, cannot create temporary table: "
           + Warehouse.getQualifiedName(tbl));
     }
 
@@ -435,7 +448,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
 
     String dbName = tbl.getDbName();
     String tblName = tbl.getTableName();
-    Map<String, Table> tables = getTempTablesForDatabase(dbName);
+    Map<String, Table> tables = getTempTablesForDatabase(dbName, tblName);
     if (tables != null && tables.containsKey(tblName)) {
       throw new MetaException(
           "Temporary table " + StatsUtils.getFullyQualifiedTableName(dbName, tblName) + " already exists");
@@ -472,7 +485,8 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
   }
 
   private org.apache.hadoop.hive.metastore.api.Table getTempTable(String dbName, String tableName) {
-    Map<String, Table> tables = getTempTablesForDatabase(dbName.toLowerCase());
+    Map<String, Table> tables = getTempTablesForDatabase(dbName.toLowerCase(),
+        tableName.toLowerCase());
     if (tables != null) {
       Table table = tables.get(tableName.toLowerCase());
       if (table != null) {
@@ -510,13 +524,13 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
 
       // Remove old temp table entry, and add new entry to list of temp tables.
       // Note that for temp tables there is no need to rename directories
-      Map<String, Table> tables = getTempTablesForDatabase(dbname);
+      Map<String, Table> tables = getTempTablesForDatabase(dbname, tbl_name);
       if (tables == null || tables.remove(tbl_name) == null) {
         throw new MetaException("Could not find temp table entry for " + dbname + "." + tbl_name);
       }
       shouldDeleteColStats = true;
 
-      tables = getTempTablesForDatabase(newDbName);
+      tables = getTempTablesForDatabase(newDbName, tbl_name);
       if (tables == null) {
         tables = new HashMap<String, Table>();
         SessionState.get().getTempTables().put(newDbName, tables);
@@ -526,7 +540,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       if (haveTableColumnsChanged(oldt, newt)) {
         shouldDeleteColStats = true;
       }
-      getTempTablesForDatabase(dbname).put(tbl_name, newTable);
+      getTempTablesForDatabase(dbname, tbl_name).put(tbl_name, newTable);
     }
 
     if (shouldDeleteColStats) {
@@ -652,7 +666,7 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     }
 
     // Remove table entry from SessionState
-    Map<String, Table> tables = getTempTablesForDatabase(dbName);
+    Map<String, Table> tables = getTempTablesForDatabase(dbName, tableName);
     if (tables == null || tables.remove(tableName) == null) {
       throw new MetaException(
           "Could not find temp table entry for " + StatsUtils.getFullyQualifiedTableName(dbName, tableName));
@@ -682,14 +696,20 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
     return newCopy;
   }
 
-  public static Map<String, Table> getTempTablesForDatabase(String dbName) {
-    return getTempTables().get(dbName);
+  /**
+   * @param dbName actual database name
+   * @param tblName actual table name or search pattern (for error message)
+   */
+  public static Map<String, Table> getTempTablesForDatabase(String dbName,
+      String tblName) {
+    return getTempTables(Warehouse.getQualifiedName(dbName, tblName)).
+        get(dbName);
   }
 
-  public static Map<String, Map<String, Table>> getTempTables() {
+  private static Map<String, Map<String, Table>> getTempTables(String msg) {
     SessionState ss = SessionState.get();
     if (ss == null) {
-      LOG.debug("No current SessionState, skipping temp tables");
+      LOG.warn("No current SessionState, skipping temp tables for " + msg);
       return Collections.emptyMap();
     }
     return ss.getTempTables();
@@ -699,7 +719,8 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       String tableName) {
     SessionState ss = SessionState.get();
     if (ss == null) {
-      LOG.debug("No current SessionState, skipping temp tables");
+      LOG.debug("No current SessionState, skipping temp tables for " +
+          Warehouse.getQualifiedName(dbName, tableName));
       return null;
     }
     String lookupName = StatsUtils.getFullyQualifiedTableName(dbName.toLowerCase(),
@@ -976,15 +997,17 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
         getQualifiedName(t.getDbName().toLowerCase(), t.getTableName().toLowerCase());
     SessionState ss = SessionState.get();
     if (ss == null) {
-      LOG.debug("No current SessionState, skipping temp partitions");
+      LOG.warn("No current SessionState, skipping temp partitions for " + qualifiedTableName);
       return null;
     }
     return ss.getTempPartitions().get(qualifiedTableName);
   }
   private static void removeTempTable(org.apache.hadoop.hive.metastore.api.Table t) {
+    String qualifiedTableName = Warehouse.
+        getQualifiedName(t.getDbName().toLowerCase(), t.getTableName().toLowerCase());
     SessionState ss = SessionState.get();
     if (ss == null) {
-      LOG.debug("No current SessionState, skipping temp partitions");
+      LOG.warn("No current SessionState, skipping temp partitions for " + qualifiedTableName);
       return;
     }
     ss.getTempPartitions().remove(Warehouse.getQualifiedName(t));
@@ -994,15 +1017,16 @@ public class SessionHiveMetaStoreClient extends HiveMetaStoreClient implements I
       //do nothing as it's not a partitioned table
       return;
     }
+    String qualifiedTableName = Warehouse.
+        getQualifiedName(t.getDbName().toLowerCase(), t.getTableName().toLowerCase());
     SessionState ss = SessionState.get();
     if (ss == null) {
-      LOG.debug("No current SessionState, skipping temp partitions");
+      LOG.warn("No current SessionState, skipping temp partitions for " + qualifiedTableName);
       return;
     }
     TempTable tt = new TempTable(t);
-    String qualifiedName = Warehouse.getQualifiedName(t);
-    if(ss.getTempPartitions().putIfAbsent(qualifiedName, tt) != null) {
-      throw new IllegalStateException("TempTable for " + qualifiedName + " already exists");
+    if(ss.getTempPartitions().putIfAbsent(qualifiedTableName, tt) != null) {
+      throw new IllegalStateException("TempTable for " + qualifiedTableName + " already exists");
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 0205650..fad0e5c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -13127,7 +13127,8 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
             throw new SemanticException("No current SessionState, cannot create temporary table "
                 + dbName + "." + tblName);
           }
-          Map<String, Table> tables = SessionHiveMetaStoreClient.getTempTablesForDatabase(dbName);
+          Map<String, Table> tables = SessionHiveMetaStoreClient.
+              getTempTablesForDatabase(dbName, tblName);
           if (tables != null && tables.containsKey(tblName)) {
             throw new SemanticException("Temporary table " + dbName + "." + tblName
                 + " already exists");

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java
index 5844f3d..3ae030d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/PartitionExport.java
@@ -20,11 +20,11 @@ package org.apache.hadoop.hive.ql.parse.repl.dump;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.PartitionIterable;
 import org.apache.hadoop.hive.ql.parse.ReplicationSpec;
 import org.apache.hadoop.hive.ql.parse.repl.dump.io.FileOperations;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,7 +35,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.hive.ql.parse.repl.dump.TableExport.AuthEntities;
 import static org.apache.hadoop.hive.ql.parse.repl.dump.TableExport.Paths;
 
 /**
@@ -49,6 +48,7 @@ class PartitionExport {
   private final String distCpDoAsUser;
   private final HiveConf hiveConf;
   private final int nThreads;
+  private final SessionState callersSession;
 
   private static final Logger LOG = LoggerFactory.getLogger(PartitionExport.class);
   private BlockingQueue<Partition> queue;
@@ -61,11 +61,14 @@ class PartitionExport {
     this.hiveConf = hiveConf;
     this.nThreads = hiveConf.getIntVar(HiveConf.ConfVars.REPL_PARTITIONS_DUMP_PARALLELISM);
     this.queue = new ArrayBlockingQueue<>(2 * nThreads);
+    this.callersSession = SessionState.get();
   }
 
   void write(final ReplicationSpec forReplicationSpec) throws InterruptedException {
-    ExecutorService producer = Executors.newFixedThreadPool(1);
+    ExecutorService producer = Executors.newFixedThreadPool(1,
+        new ThreadFactoryBuilder().setNameFormat("partition-submitter-thread-%d").build());
     producer.submit(() -> {
+      SessionState.setCurrentSessionState(callersSession);
       for (Partition partition : partitionIterable) {
         try {
           queue.put(partition);

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
index 60b63d4..27f0406 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
@@ -41,6 +41,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.commons.lang.StringUtils;
@@ -117,10 +118,14 @@ public class SessionState {
   static final String LOCK_FILE_NAME = "inuse.lck";
   static final String INFO_FILE_NAME = "inuse.info";
 
-  private final Map<String, Map<String, Table>> tempTables = new HashMap<>();
+  /**
+   * Concurrent since SessionState is often propagated to workers in thread pools
+   */
+  private final Map<String, Map<String, Table>> tempTables = new ConcurrentHashMap<>();
   private final Map<String, Map<String, ColumnStatisticsObj>> tempTableColStats =
-      new HashMap<String, Map<String, ColumnStatisticsObj>>();
-  private final Map<String, SessionHiveMetaStoreClient.TempTable> tempPartitions = new HashMap<>();
+      new ConcurrentHashMap<>();
+  private final Map<String, SessionHiveMetaStoreClient.TempTable> tempPartitions =
+      new ConcurrentHashMap<>();
 
   protected ClassLoader parentLoader;
 
@@ -537,8 +542,7 @@ public class SessionState {
    * Singleton Session object per thread.
    *
    **/
-  private static InheritableThreadLocal<SessionStates> tss =
-      new InheritableThreadLocal<SessionStates>() {
+  private static ThreadLocal<SessionStates> tss = new ThreadLocal<SessionStates>() {
     @Override
     protected SessionStates initialValue() {
       return new SessionStates();

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/avro_non_nullable_union.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/avro_non_nullable_union.q.out b/ql/src/test/results/clientnegative/avro_non_nullable_union.q.out
index 0149e9f..c933081 100644
--- a/ql/src/test/results/clientnegative/avro_non_nullable_union.q.out
+++ b/ql/src/test/results/clientnegative/avro_non_nullable_union.q.out
@@ -35,5 +35,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@union_nullable_test_text
 PREHOOK: Output: default@union_non_nullable_test_avro
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/cachingprintstream.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/cachingprintstream.q.out b/ql/src/test/results/clientnegative/cachingprintstream.q.out
index 107811b..0acb772 100644
--- a/ql/src/test/results/clientnegative/cachingprintstream.q.out
+++ b/ql/src/test/results/clientnegative/cachingprintstream.q.out
@@ -8,12 +8,10 @@ PREHOOK: query: FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 Begin cached logs.
 PREHOOK: query: FROM src SELECT TRANSFORM (key, value) USING 'FAKE_SCRIPT_SHOULD_NOT_EXIST' AS key, value
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 End cached logs.
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/check_constraint_tbl_level.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/check_constraint_tbl_level.q.out b/ql/src/test/results/clientnegative/check_constraint_tbl_level.q.out
index c18f733..3ed9c25 100644
--- a/ql/src/test/results/clientnegative/check_constraint_tbl_level.q.out
+++ b/ql/src/test/results/clientnegative/check_constraint_tbl_level.q.out
@@ -11,5 +11,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@tti
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/check_constraint_violation.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/check_constraint_violation.q.out b/ql/src/test/results/clientnegative/check_constraint_violation.q.out
index 05e3ff5..82f367c 100644
--- a/ql/src/test/results/clientnegative/check_constraint_violation.q.out
+++ b/ql/src/test/results/clientnegative/check_constraint_violation.q.out
@@ -11,5 +11,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: _dummy_database@_dummy_table
 PREHOOK: Output: default@tti
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/compute_stats_long.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/compute_stats_long.q.out b/ql/src/test/results/clientnegative/compute_stats_long.q.out
index 0f7cbd6..79f2146 100644
--- a/ql/src/test/results/clientnegative/compute_stats_long.q.out
+++ b/ql/src/test/results/clientnegative/compute_stats_long.q.out
@@ -18,5 +18,4 @@ PREHOOK: query: select compute_stats(a, 10000) from tab_int
 PREHOOK: type: QUERY
 PREHOOK: Input: default@tab_int
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/dyn_part3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/dyn_part3.q.out b/ql/src/test/results/clientnegative/dyn_part3.q.out
index 6bcd0b4..b9f1442 100644
--- a/ql/src/test/results/clientnegative/dyn_part3.q.out
+++ b/ql/src/test/results/clientnegative/dyn_part3.q.out
@@ -11,5 +11,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@nzhang_part
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out b/ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out
index b9b6a3a..44d266c 100644
--- a/ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out
+++ b/ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out
@@ -20,5 +20,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@max_parts
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out b/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out
index 064c44d..a242366 100644
--- a/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out
+++ b/ql/src/test/results/clientnegative/dynamic_partitions_with_whitelist.q.out
@@ -29,5 +29,4 @@ PREHOOK: Input: default@source_table
 PREHOOK: Input: default@source_table@ds=2008-04-08/hr=11
 PREHOOK: Output: default@dest_table
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/insertsel_fail.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/insertsel_fail.q.out b/ql/src/test/results/clientnegative/insertsel_fail.q.out
index 347e1b1..f8cc704 100644
--- a/ql/src/test/results/clientnegative/insertsel_fail.q.out
+++ b/ql/src/test/results/clientnegative/insertsel_fail.q.out
@@ -1 +1 @@
-FAILED: ParseException line 2:72 extraneous input 'a' expecting EOF near '<EOF>'
+FAILED: ParseException line 1:72 extraneous input 'a' expecting EOF near '<EOF>'

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out b/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
index 8232b5f..238f342 100644
--- a/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
+++ b/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
@@ -3,5 +3,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: hdfs://### HDFS PATH ###
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/merge_constraint_notnull.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/merge_constraint_notnull.q.out b/ql/src/test/results/clientnegative/merge_constraint_notnull.q.out
index 0fd26ff..088e249 100644
--- a/ql/src/test/results/clientnegative/merge_constraint_notnull.q.out
+++ b/ql/src/test/results/clientnegative/merge_constraint_notnull.q.out
@@ -52,5 +52,4 @@ PREHOOK: Output: default@testt
 PREHOOK: Output: default@testt
 PREHOOK: Output: default@testt
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/script_broken_pipe2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/script_broken_pipe2.q.out b/ql/src/test/results/clientnegative/script_broken_pipe2.q.out
index ca022a0..2202bf0 100644
--- a/ql/src/test/results/clientnegative/script_broken_pipe2.q.out
+++ b/ql/src/test/results/clientnegative/script_broken_pipe2.q.out
@@ -2,5 +2,4 @@ PREHOOK: query: SELECT TRANSFORM(key, value, key, value, key, value, key, value,
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/script_broken_pipe3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/script_broken_pipe3.q.out b/ql/src/test/results/clientnegative/script_broken_pipe3.q.out
index 41d0cf3..21415a6 100644
--- a/ql/src/test/results/clientnegative/script_broken_pipe3.q.out
+++ b/ql/src/test/results/clientnegative/script_broken_pipe3.q.out
@@ -2,5 +2,4 @@ PREHOOK: query: SELECT TRANSFORM(*) USING 'false' AS a, b FROM (SELECT TRANSFORM
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/script_error.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/script_error.q.out b/ql/src/test/results/clientnegative/script_error.q.out
index c8a905a..c38f6d4 100644
--- a/ql/src/test/results/clientnegative/script_error.q.out
+++ b/ql/src/test/results/clientnegative/script_error.q.out
@@ -47,5 +47,4 @@ FROM src
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/serde_regex2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/serde_regex2.q.out b/ql/src/test/results/clientnegative/serde_regex2.q.out
index 5c8d943..b0703fb 100644
--- a/ql/src/test/results/clientnegative/serde_regex2.q.out
+++ b/ql/src/test/results/clientnegative/serde_regex2.q.out
@@ -60,5 +60,4 @@ PREHOOK: query: SELECT * FROM serde_regex ORDER BY `time`
 PREHOOK: type: QUERY
 PREHOOK: Input: default@serde_regex
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out b/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out
index a152311..70d2ee4 100644
--- a/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out
+++ b/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out
@@ -22,5 +22,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@tmptable
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out b/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out
index 81d3010..91f9090 100644
--- a/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out
+++ b/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out
@@ -21,5 +21,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@tmptable
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out b/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out
index a152311..70d2ee4 100644
--- a/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out
+++ b/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out
@@ -22,5 +22,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 PREHOOK: Output: default@tmptable
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out b/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out
index a739157..ff17cd0 100644
--- a/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out
+++ b/ql/src/test/results/clientnegative/subquery_corr_in_agg.q.out
@@ -40,5 +40,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@part1
 PREHOOK: Input: default@supply
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/subquery_in_implicit_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/subquery_in_implicit_gby.q.out b/ql/src/test/results/clientnegative/subquery_in_implicit_gby.q.out
index 0630a04..5dc55c4 100644
--- a/ql/src/test/results/clientnegative/subquery_in_implicit_gby.q.out
+++ b/ql/src/test/results/clientnegative/subquery_in_implicit_gby.q.out
@@ -39,5 +39,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@t
 PREHOOK: Input: default@tt
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/subquery_notin_implicit_gby.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/subquery_notin_implicit_gby.q.out b/ql/src/test/results/clientnegative/subquery_notin_implicit_gby.q.out
index f6aeada..db6d0a6 100644
--- a/ql/src/test/results/clientnegative/subquery_notin_implicit_gby.q.out
+++ b/ql/src/test/results/clientnegative/subquery_notin_implicit_gby.q.out
@@ -39,5 +39,4 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@t
 PREHOOK: Input: default@tt
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/subquery_scalar_corr_multi_rows.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/subquery_scalar_corr_multi_rows.q.out b/ql/src/test/results/clientnegative/subquery_scalar_corr_multi_rows.q.out
index 96dfbc2..8377085 100644
--- a/ql/src/test/results/clientnegative/subquery_scalar_corr_multi_rows.q.out
+++ b/ql/src/test/results/clientnegative/subquery_scalar_corr_multi_rows.q.out
@@ -3,5 +3,4 @@ PREHOOK: query: select * from part where p_size >
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out b/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out
index ff377b9..0a780db 100644
--- a/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out
+++ b/ql/src/test/results/clientnegative/subquery_scalar_multi_rows.q.out
@@ -3,5 +3,4 @@ PREHOOK: query: select p_name from part where p_size > (select p_size from part)
 PREHOOK: type: QUERY
 PREHOOK: Input: default@part
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/udf_assert_true.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/udf_assert_true.q.out b/ql/src/test/results/clientnegative/udf_assert_true.q.out
index bd4355c..7fc50d6 100644
--- a/ql/src/test/results/clientnegative/udf_assert_true.q.out
+++ b/ql/src/test/results/clientnegative/udf_assert_true.q.out
@@ -151,5 +151,4 @@ PREHOOK: query: SELECT ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRAY(1,
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/udf_assert_true2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/udf_assert_true2.q.out b/ql/src/test/results/clientnegative/udf_assert_true2.q.out
index 5c4d67a..f061a92 100644
--- a/ql/src/test/results/clientnegative/udf_assert_true2.q.out
+++ b/ql/src/test/results/clientnegative/udf_assert_true2.q.out
@@ -69,5 +69,4 @@ PREHOOK: query: SELECT 1 + ASSERT_TRUE(x < 2) FROM src LATERAL VIEW EXPLODE(ARRA
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/udf_reflect_neg.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/udf_reflect_neg.q.out b/ql/src/test/results/clientnegative/udf_reflect_neg.q.out
index 73d2afa..7ae91b5 100644
--- a/ql/src/test/results/clientnegative/udf_reflect_neg.q.out
+++ b/ql/src/test/results/clientnegative/udf_reflect_neg.q.out
@@ -9,5 +9,4 @@ FROM src LIMIT 1
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/udf_test_error.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/udf_test_error.q.out b/ql/src/test/results/clientnegative/udf_test_error.q.out
index 41e6e02..5ad5ff6 100644
--- a/ql/src/test/results/clientnegative/udf_test_error.q.out
+++ b/ql/src/test/results/clientnegative/udf_test_error.q.out
@@ -8,5 +8,4 @@ PREHOOK: query: SELECT test_error(key < 125 OR key > 130) FROM src
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask

http://git-wip-us.apache.org/repos/asf/hive/blob/ee6a5326/ql/src/test/results/clientnegative/udf_test_error_reduce.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/udf_test_error_reduce.q.out b/ql/src/test/results/clientnegative/udf_test_error_reduce.q.out
index d5d53cc..fec6b5d 100644
--- a/ql/src/test/results/clientnegative/udf_test_error_reduce.q.out
+++ b/ql/src/test/results/clientnegative/udf_test_error_reduce.q.out
@@ -13,5 +13,4 @@ FROM (
 PREHOOK: type: QUERY
 PREHOOK: Input: default@src
 #### A masked pattern was here ####
-Error during job, obtaining debugging information...
 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask