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 <ekoif...@apache.org> Authored: Mon Apr 30 17:53:45 2018 -0700 Committer: Eugene Koifman <ekoif...@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