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 2012/07/26 19:41:17 UTC

svn commit: r1366103 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ conf/ ql/src/java/org/apache/hadoop/hive/ql/ ql/src/java/org/apache/hadoop/hive/ql/exec/ ql/src/java/org/apache/hadoop/hive/ql/optimizer/ ql/src/java/org/apache/hadoop/...

Author: kevinwilfong
Date: Thu Jul 26 17:41:15 2012
New Revision: 1366103

URL: http://svn.apache.org/viewvc?rev=1366103&view=rev
Log:
HIVE-1653. Ability to enforce correct stats. (njain via kevinwilfong)

Added:
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java
    hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q
    hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q
    hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_1.q
    hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_2.q
    hive/trunk/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q
    hive/trunk/ql/src/test/queries/clientpositive/stats_publisher_error_1.q
    hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_1.q.out
    hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out
    hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out
    hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out
    hive/trunk/ql/src/test/results/clientpositive/stats_aggregator_error_1.q.out
    hive/trunk/ql/src/test/results/clientpositive/stats_publisher_error_1.q.out
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/StatsWork.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Thu Jul 26 17:41:15 2012
@@ -517,6 +517,7 @@ public class HiveConf extends Configurat
     // should the raw data size be collected when analayzing tables
     CLIENT_STATS_COUNTERS("hive.client.stats.counters", ""),
     //Subset of counters that should be of interest for hive.client.stats.publishers (when one wants to limit their publishing). Non-display names should be used".
+    HIVE_STATS_RELIABLE("hive.stats.reliable", false),
 
     // Concurrency
     HIVE_SUPPORT_CONCURRENCY("hive.support.concurrency", false),
@@ -701,11 +702,16 @@ public class HiveConf extends Configurat
     }
 
     enum VarType {
-      STRING { void checkType(String value) throws Exception { } },
-      INT { void checkType(String value) throws Exception { Integer.valueOf(value); } },
-      LONG { void checkType(String value) throws Exception { Long.valueOf(value); } },
-      FLOAT { void checkType(String value) throws Exception { Float.valueOf(value); } },
-      BOOLEAN { void checkType(String value) throws Exception { Boolean.valueOf(value); } };
+      STRING { @Override
+      void checkType(String value) throws Exception { } },
+      INT { @Override
+      void checkType(String value) throws Exception { Integer.valueOf(value); } },
+      LONG { @Override
+      void checkType(String value) throws Exception { Long.valueOf(value); } },
+      FLOAT { @Override
+      void checkType(String value) throws Exception { Float.valueOf(value); } },
+      BOOLEAN { @Override
+      void checkType(String value) throws Exception { Boolean.valueOf(value); } };
 
       boolean isType(String value) {
         try { checkType(value); } catch (Exception e) { return false; }

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Thu Jul 26 17:41:15 2012
@@ -944,6 +944,15 @@
 </property>
 
 <property>
+  <name>hive.stats.reliable</name>
+  <value>false</value>
+  <description>Whether queries will fail because stats cannot be collected completely accurately. 
+    If this is set to true, reading/writing from/into a partition may fail becuase the stats 
+    could not be computed accurately.
+  </description>
+</property>
+
+<property>
   <name>hive.support.concurrency</name>
   <value>false</value>
   <description>Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks.</description>

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java Thu Jul 26 17:41:15 2012
@@ -29,7 +29,8 @@ import org.apache.hadoop.hive.ql.parse.A
 import org.apache.hadoop.hive.ql.parse.ASTNodeOrigin;
 
 /**
- * List of error messages thrown by the parser.
+ * List of all error messages.
+ * This list contains both compile time and run-time errors.
  **/
 
 public enum ErrorMsg {
@@ -226,7 +227,29 @@ public enum ErrorMsg {
   SCRIPT_GENERIC_ERROR(20002, "Hive encountered some unknown error while "
       + "running your custom script."),
   SCRIPT_CLOSING_ERROR(20003, "An error occurred when trying to close the Operator " +
-      "running your custom script.")
+      "running your custom script."),
+
+  STATSPUBLISHER_NOT_OBTAINED(30000, "StatsPublisher cannot be obtained. " +
+    "There was a error to retrieve the StatsPublisher, and retrying " +
+    "might help. If you dont want the query to fail because accurate statistics " +
+    "could not be collected, set hive.stats.reliable=false"),
+  STATSPUBLISHER_INITIALIZATION_ERROR(30001, "StatsPublisher cannot be initialized. " +
+    "There was a error in the initialization of StatsPublisher, and retrying " +
+    "might help. If you dont want the query to fail because accurate statistics " +
+    "could not be collected, set hive.stats.reliable=false"),
+  STATSPUBLISHER_CONNECTION_ERROR(30002, "StatsPublisher cannot be connected to." +
+    "There was a error while connecting to the StatsPublisher, and retrying " +
+    "might help. If you dont want the query to fail because accurate statistics " +
+    "could not be collected, set hive.stats.reliable=false"),
+  STATSPUBLISHER_PUBLISHING_ERROR(30003, "Error in publishing stats. There was an " +
+    "error in publishing stats via StatsPublisher, and retrying " +
+    "might help. If you dont want the query to fail because accurate statistics " +
+    "could not be collected, set hive.stats.reliable=false"),
+  STATSPUBLISHER_CLOSING_ERROR(30004, "StatsPublisher cannot be closed." +
+    "There was a error while closing the StatsPublisher, and retrying " +
+    "might help. If you dont want the query to fail because accurate statistics " +
+    "could not be collected, set hive.stats.reliable=false"),
+
     ;
 
   private int errorCode;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ExecDriver.java Thu Jul 26 17:41:15 2012
@@ -50,6 +50,7 @@ import org.apache.hadoop.hive.common.Com
 import org.apache.hadoop.hive.common.LogUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -425,7 +426,12 @@ public class ExecDriver extends Task<Map
         String statsImplementationClass = HiveConf.getVar(job, HiveConf.ConfVars.HIVESTATSDBCLASS);
         if (StatsFactory.setImplementation(statsImplementationClass, job)) {
           statsPublisher = StatsFactory.getStatsPublisher();
-          statsPublisher.init(job); // creating stats table if not exists
+          if (!statsPublisher.init(job)) { // creating stats table if not exists
+            if (conf.getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE)) {
+              throw
+                new HiveException(ErrorMsg.STATSPUBLISHER_INITIALIZATION_ERROR.getErrorCodedMsg());
+            }
+          }
         }
       }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java Thu Jul 26 17:41:15 2012
@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils;
 import org.apache.hadoop.hive.ql.io.HiveKey;
 import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
@@ -704,7 +705,6 @@ public class FileSinkOperator extends Te
 
   @Override
   public void closeOp(boolean abort) throws HiveException {
-
     if (!bDynParts && !filesCreated) {
       createBucketFiles(fsp);
     }
@@ -787,18 +787,27 @@ public class FileSinkOperator extends Te
     }
   }
 
-  private void publishStats() {
+  private void publishStats() throws HiveException {
+    boolean isStatsReliable = conf.isStatsReliable();
+
     // Initializing a stats publisher
     StatsPublisher statsPublisher = Utilities.getStatsPublisher(jc);
 
     if (statsPublisher == null) {
       // just return, stats gathering should not block the main query
       LOG.error("StatsPublishing error: StatsPublisher is not initialized.");
+      if (isStatsReliable) {
+        throw new HiveException(ErrorMsg.STATSPUBLISHER_NOT_OBTAINED.getErrorCodedMsg());
+      }
       return;
     }
+
     if (!statsPublisher.connect(hconf)) {
       // just return, stats gathering should not block the main query
       LOG.error("StatsPublishing error: cannot connect to database");
+      if (isStatsReliable) {
+        throw new HiveException(ErrorMsg.STATSPUBLISHER_CONNECTION_ERROR.getErrorCodedMsg());
+      }
       return;
     }
 
@@ -823,8 +832,20 @@ public class FileSinkOperator extends Te
       for (String statType : fspValue.stat.getStoredStats()) {
         statsToPublish.put(statType, Long.toString(fspValue.stat.getStat(statType)));
       }
-      statsPublisher.publishStat(key, statsToPublish);
+      if (!statsPublisher.publishStat(key, statsToPublish)) {
+        // The original exception is lost.
+        // Not changing the interface to maintain backward compatibility
+        if (isStatsReliable) {
+          throw new HiveException(ErrorMsg.STATSPUBLISHER_PUBLISHING_ERROR.getErrorCodedMsg());
+        }
+      }
+    }
+    if (!statsPublisher.closeConnection()) {
+      // The original exception is lost.
+      // Not changing the interface to maintain backward compatibility
+      if (isStatsReliable) {
+        throw new HiveException(ErrorMsg.STATSPUBLISHER_CLOSING_ERROR.getErrorCodedMsg());
+      }
     }
-    statsPublisher.closeConnection();
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/StatsTask.java Thu Jul 26 17:41:15 2012
@@ -218,12 +218,15 @@ public class StatsTask extends Task<Stat
     LOG.info("Executing stats task");
     // Make sure that it is either an ANALYZE, INSERT OVERWRITE or CTAS command
     short workComponentsPresent = 0;
-    if (work.getLoadTableDesc() != null)
+    if (work.getLoadTableDesc() != null) {
       workComponentsPresent++;
-    if (work.getTableSpecs() != null)
+    }
+    if (work.getTableSpecs() != null) {
       workComponentsPresent++;
-    if (work.getLoadFileDesc() != null)
+    }
+    if (work.getLoadFileDesc() != null) {
       workComponentsPresent++;
+    }
 
     assert (workComponentsPresent == 1);
 
@@ -266,6 +269,7 @@ public class StatsTask extends Task<Stat
   private int aggregateStats() {
 
     StatsAggregator statsAggregator = null;
+    int ret = 0;
 
     try {
       // Stats setup:
@@ -426,17 +430,22 @@ public class StatsTask extends Task<Stat
       console.printInfo("Table " + tableFullName + " stats: [" + tblStats.toString() + ']');
 
     } catch (Exception e) {
-      // return 0 since StatsTask should not fail the whole job
       console.printInfo("[Warning] could not update stats.",
           "Failed with exception " + e.getMessage() + "\n"
               + StringUtils.stringifyException(e));
+
+      // Fail the query if the stats are supposed to be reliable
+      if (work.isStatsReliable()) {
+        ret = 1;
+      }
     } finally {
       if (statsAggregator != null) {
         statsAggregator.closeConnection();
       }
     }
-    // StatsTask always return 0 so that the whole job won't fail
-    return 0;
+    // The return value of 0 indicates success,
+    // anything else indicates failure
+    return ret;
   }
 
   private boolean existStats(Map<String, String> parameters) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/TableScanOperator.java Thu Jul 26 17:41:15 2012
@@ -27,6 +27,7 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
@@ -91,7 +92,6 @@ public class TableScanOperator extends O
 
   private void gatherStats(Object row) {
     // first row/call or a new partition
-
     if ((currentStat == null) || inputFileChanged) {
       inputFileChanged = false;
       if (conf.getPartColumns() == null || conf.getPartColumns().size() == 0) {
@@ -230,12 +230,17 @@ public class TableScanOperator extends O
     return OperatorType.TABLESCAN;
   }
 
-  private void publishStats() {
+  private void publishStats() throws HiveException {
+    boolean isStatsReliable = conf.isStatsReliable();
+
     // Initializing a stats publisher
     StatsPublisher statsPublisher = Utilities.getStatsPublisher(jc);
     if (!statsPublisher.connect(jc)) {
       // just return, stats gathering should not block the main query.
       LOG.info("StatsPublishing error: cannot connect to database.");
+      if (isStatsReliable) {
+        throw new HiveException(ErrorMsg.STATSPUBLISHER_CONNECTION_ERROR.getErrorCodedMsg());
+      }
       return;
     }
 
@@ -257,9 +262,17 @@ public class TableScanOperator extends O
       for(String statType : stats.get(pspecs).getStoredStats()) {
         statsToPublish.put(statType, Long.toString(stats.get(pspecs).getStat(statType)));
       }
-      statsPublisher.publishStat(key, statsToPublish);
+      if (!statsPublisher.publishStat(key, statsToPublish)) {
+        if (isStatsReliable) {
+          throw new HiveException(ErrorMsg.STATSPUBLISHER_PUBLISHING_ERROR.getErrorCodedMsg());
+        }
+      }
       LOG.info("publishing : " + key + " : " + statsToPublish.toString());
     }
-    statsPublisher.closeConnection();
+    if (!statsPublisher.closeConnection()) {
+      if (isStatsReliable) {
+        throw new HiveException(ErrorMsg.STATSPUBLISHER_CLOSING_ERROR.getErrorCodedMsg());
+      }
+    }
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRFileSink1.java Thu Jul 26 17:41:15 2012
@@ -175,7 +175,7 @@ public class GenMRFileSink1 implements N
        statsWork = new StatsWork(mvWork.getLoadFileWork());
     }
     assert statsWork != null : "Error when genereting StatsTask";
-
+    statsWork.setStatsReliable(hconf.getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE));
     MapredWork mrWork = (MapredWork) currTask.getWork();
 
     // AggKey in StatsWork is used for stats aggregation while StatsAggPrefix
@@ -186,6 +186,7 @@ public class GenMRFileSink1 implements N
     // mark the MapredWork and FileSinkOperator for gathering stats
     nd.getConf().setGatherStats(true);
     mrWork.setGatheringStats(true);
+    nd.getConf().setStatsReliable(hconf.getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE));
     // mrWork.addDestinationTable(nd.getConf().getTableInfo().getTableName());
 
     // subscribe feeds from the MoveTask so that MoveTask can forward the list

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GenMRTableScan1.java Thu Jul 26 17:41:15 2012
@@ -24,6 +24,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Stack;
 
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -34,11 +35,11 @@ import org.apache.hadoop.hive.ql.lib.Nod
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.optimizer.GenMRProcContext.GenMapRedCtx;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.tableSpec;
 import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.QBParseInfo;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.tableSpec;
 import org.apache.hadoop.hive.ql.plan.MapredWork;
 import org.apache.hadoop.hive.ql.plan.StatsWork;
 /**
@@ -86,6 +87,8 @@ public class GenMRTableScan1 implements 
 
           StatsWork statsWork = new StatsWork(parseCtx.getQB().getParseInfo().getTableSpec());
           statsWork.setAggKey(op.getConf().getStatsAggPrefix());
+          statsWork.setStatsReliable(
+            parseCtx.getConf().getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE));
           Task<StatsWork> statsTask = TaskFactory.get(statsWork, parseCtx.getConf());
           currTask.addDependentTask(statsTask);
           if (!ctx.getRootTasks().contains(currTask)) {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Thu Jul 26 17:41:15 2012
@@ -1243,6 +1243,7 @@ public class DDLSemanticAnalyzer extends
           statDesc = new StatsWork(ltd);
         }
         statDesc.setNoStatsAggregator(true);
+        statDesc.setStatsReliable(conf.getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE));
         Task<? extends Serializable> statTask = TaskFactory.get(statDesc, conf);
         moveTsk.addDependentTask(statTask);
       }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java Thu Jul 26 17:41:15 2012
@@ -6685,6 +6685,7 @@ public class SemanticAnalyzer extends Ba
       tsDesc.setGatherStats(false);
     } else {
       tsDesc.setGatherStats(true);
+      tsDesc.setStatsReliable(conf.getBoolVar(HiveConf.ConfVars.HIVE_STATS_RELIABLE));
 
       // append additional virtual columns for storing statistics
       Iterator<VirtualColumn> vcs = VirtualColumn.getStatsRegistry(conf).iterator();

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java Thu Jul 26 17:41:15 2012
@@ -46,6 +46,7 @@ public class FileSinkDesc implements Ser
   private DynamicPartitionCtx dpCtx;
   private String staticSpec; // static partition spec ends with a '/'
   private boolean gatherStats;
+  private boolean statsReliable;
 
   public FileSinkDesc() {
   }
@@ -247,4 +248,12 @@ public class FileSinkDesc implements Ser
       statsKeyPref = k + Path.SEPARATOR;
     }
   }
+
+  public boolean isStatsReliable() {
+    return statsReliable;
+  }
+
+  public void setStatsReliable(boolean statsReliable) {
+    this.statsReliable = statsReliable;
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/StatsWork.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/StatsWork.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/StatsWork.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/StatsWork.java Thu Jul 26 17:41:15 2012
@@ -34,6 +34,7 @@ public class StatsWork implements Serial
   private LoadTableDesc loadTableDesc; // same as MoveWork.loadTableDesc -- for FileSinkOperator
   private LoadFileDesc loadFileDesc;   // same as MoveWork.loadFileDesc -- for FileSinkOperator
   private String aggKey;               // aggregation key prefix
+  private boolean statsReliable;     // are stats completely reliable
 
   private boolean noStatsAggregator = false;
 
@@ -52,6 +53,10 @@ public class StatsWork implements Serial
     this.loadFileDesc = loadFileDesc;
   }
 
+  public StatsWork(boolean statsReliable) {
+    this.statsReliable = statsReliable;
+  }
+
   public tableSpec getTableSpecs() {
     return tableSpecs;
   }
@@ -81,4 +86,11 @@ public class StatsWork implements Serial
     this.noStatsAggregator = noStatsAggregator;
   }
 
+  public boolean isStatsReliable() {
+    return statsReliable;
+  }
+
+  public void setStatsReliable(boolean statsReliable) {
+    this.statsReliable = statsReliable;
+  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java?rev=1366103&r1=1366102&r2=1366103&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java Thu Jul 26 17:41:15 2012
@@ -48,6 +48,7 @@ public class TableScanDesc implements Se
    *
    */
   private boolean gatherStats;
+  private boolean statsReliable;
 
   private ExprNodeDesc filterExpr;
 
@@ -125,4 +126,12 @@ public class TableScanDesc implements Se
   public String getStatsAggPrefix() {
     return statsAggKeyPrefix;
   }
+
+  public boolean isStatsReliable() {
+    return statsReliable;
+  }
+
+  public void setStatsReliable(boolean statsReliable) {
+    this.statsReliable = statsReliable;
+  }
 }

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java Thu Jul 26 17:41:15 2012
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.stats;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * An test implementation for StatsAggregator.
+ * The method corresponding to the configuration parameter
+ * hive.test.dummystats.aggregator fail, whereas all
+ * other methods succeed.
+ */
+
+public class DummyStatsAggregator implements StatsAggregator {
+  String errorMethod = null;
+
+  // This is a test. The parameter hive.test.dummystats.aggregator's value
+  // denotes the method which needs to throw an error.
+  public boolean connect(Configuration hconf) {
+    errorMethod = hconf.get("hive.test.dummystats.aggregator", "");
+    if (errorMethod.equalsIgnoreCase("connect")) {
+      return false;
+    }
+
+    return true;
+  }
+
+  public String aggregateStats(String keyPrefix, String statType) {
+    return null;
+  }
+
+  public boolean closeConnection() {
+    if (errorMethod.equalsIgnoreCase("closeConnection")) {
+      return false;
+    }
+    return true;
+  }
+
+  public boolean cleanUp(String keyPrefix) {
+    if (errorMethod.equalsIgnoreCase("cleanUp")) {
+      return false;
+    }
+    return true;
+  }
+}

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java Thu Jul 26 17:41:15 2012
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.stats;
+
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * An test implementation for StatsPublisher.
+ * The method corresponding to the configuration parameter
+ * hive.test.dummystats.publisher fail, whereas all
+ * other methods succeed
+ */
+
+public class DummyStatsPublisher implements StatsPublisher {
+
+  String errorMethod = null;
+
+  // This is a test. The parameter hive.test.dummystats.publisher's value
+  // denotes the method which needs to throw an error.
+  public boolean init(Configuration hconf) {
+    errorMethod = hconf.get("hive.test.dummystats.publisher", "");
+    if (errorMethod.equalsIgnoreCase("init")) {
+      return false;
+    }
+
+    return true;
+  }
+
+  public boolean connect(Configuration hconf) {
+    errorMethod = hconf.get("hive.test.dummystats.publisher", "");
+    if (errorMethod.equalsIgnoreCase("connect")) {
+      return false;
+    }
+
+    return true;
+  }
+
+  public boolean publishStat(String fileID, Map<String, String> stats) {
+    if (errorMethod.equalsIgnoreCase("publishStat")) {
+      return false;
+    }
+    return true;
+  }
+
+  public boolean closeConnection() {
+    if (errorMethod.equalsIgnoreCase("closeConnection")) {
+      return false;
+    }
+    return true;
+  }
+}

Added: hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q Thu Jul 26 17:41:15 2012
@@ -0,0 +1,18 @@
+-- In this test, there is a dummy stats aggregator which throws an error when the
+-- method connect is called (as indicated by the parameter hive.test.dummystats.aggregator)
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string);
+
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
+set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
+set hive.test.dummystats.aggregator=connect;
+
+set hive.stats.reliable=false;
+INSERT OVERWRITE TABLE tmptable select * from src;
+
+set hive.stats.reliable=true;
+INSERT OVERWRITE TABLE tmptable select * from src;

Added: hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q Thu Jul 26 17:41:15 2012
@@ -0,0 +1,16 @@
+-- In this test, the stats aggregator does not exists.
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string);
+
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
+set hive.stats.default.aggregator="";
+
+set hive.stats.reliable=false;
+INSERT OVERWRITE TABLE tmptable select * from src;
+
+set hive.stats.reliable=true;
+INSERT OVERWRITE TABLE tmptable select * from src;

Added: hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_1.q?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_1.q Thu Jul 26 17:41:15 2012
@@ -0,0 +1,18 @@
+-- In this test, there is a dummy stats publisher which throws an error when the
+-- method connect is called (as indicated by the parameter hive.test.dummystats.publisher)
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string);
+
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
+set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
+set hive.test.dummystats.publisher=connect;
+
+set hive.stats.reliable=false;
+INSERT OVERWRITE TABLE tmptable select * from src;
+
+set hive.stats.reliable=true;
+INSERT OVERWRITE TABLE tmptable select * from src;

Added: hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_2.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_2.q?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_2.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/stats_publisher_error_2.q Thu Jul 26 17:41:15 2012
@@ -0,0 +1,16 @@
+-- In this test, the stats publisher does not exists.
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string);
+
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher="";
+set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
+
+set hive.stats.reliable=false;
+INSERT OVERWRITE TABLE tmptable select * from src;
+
+set hive.stats.reliable=true;
+INSERT OVERWRITE TABLE tmptable select * from src;

Added: hive/trunk/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q Thu Jul 26 17:41:15 2012
@@ -0,0 +1,29 @@
+-- In this test, there is a dummy stats aggregator which throws an error when various
+-- methods are called (as indicated by the parameter hive.test.dummystats.agregator)
+-- Since stats need not be reliable (by setting hive.stats.reliable to false), the 
+-- insert statements succeed. The insert statement succeeds even if the stats aggregator
+-- is set to null, since stats need not be reliable.
+
+create table tmptable(key string, value string);
+
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
+set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
+set hive.stats.reliable=false;
+
+set hive.test.dummystats.aggregator=connect;
+
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;
+
+set hive.test.dummystats.aggregator=closeConnection;
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;
+
+set hive.test.dummystats.aggregator=cleanUp;
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;
+
+set hive.stats.default.aggregator="";
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;

Added: hive/trunk/ql/src/test/queries/clientpositive/stats_publisher_error_1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/stats_publisher_error_1.q?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/stats_publisher_error_1.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/stats_publisher_error_1.q Thu Jul 26 17:41:15 2012
@@ -0,0 +1,29 @@
+-- In this test, there is a dummy stats publisher which throws an error when various
+-- methods are called (as indicated by the parameter hive.test.dummystats.publisher)
+-- Since stats need not be reliable (by setting hive.stats.reliable to false), the
+-- insert statements succeed. The insert statement succeeds even if the stats publisher
+-- is set to null, since stats need not be reliable.
+
+create table tmptable(key string, value string);
+
+set hive.stats.dbclass=dummy;
+set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
+set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
+set hive.stats.reliable=false;
+
+set hive.test.dummystats.publisher=connect;
+
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;
+
+set hive.test.dummystats.publisher=publishStat;
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;
+
+set hive.test.dummystats.publisher=closeConnection;
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;
+
+set hive.stats.default.publisher="";
+INSERT OVERWRITE TABLE tmptable select * from src;
+select count(1) from tmptable;

Added: hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_1.q.out?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_1.q.out Thu Jul 26 17:41:15 2012
@@ -0,0 +1,32 @@
+PREHOOK: query: -- In this test, there is a dummy stats aggregator which throws an error when the
+-- method connect is called (as indicated by the parameter hive.test.dummystats.aggregator)
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- In this test, there is a dummy stats aggregator which throws an error when the
+-- method connect is called (as indicated by the parameter hive.test.dummystats.aggregator)
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tmptable
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.StatsTask

Added: hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out Thu Jul 26 17:41:15 2012
@@ -0,0 +1,40 @@
+PREHOOK: query: -- In this test, the stats aggregator does not exists.
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- In this test, the stats aggregator does not exists.
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tmptable
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+Execution failed with exit status: 2
+Obtaining error information
+
+Task failed!
+Task ID:
+  Stage-1
+
+Logs:
+
+#### A masked pattern was here ####
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask

Added: hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out Thu Jul 26 17:41:15 2012
@@ -0,0 +1,42 @@
+PREHOOK: query: -- In this test, there is a dummy stats publisher which throws an error when the
+-- method connect is called (as indicated by the parameter hive.test.dummystats.publisher)
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- In this test, there is a dummy stats publisher which throws an error when the
+-- method connect is called (as indicated by the parameter hive.test.dummystats.publisher)
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tmptable
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+Execution failed with exit status: 2
+Obtaining error information
+
+Task failed!
+Task ID:
+  Stage-1
+
+Logs:
+
+#### A masked pattern was here ####
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask

Added: hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out Thu Jul 26 17:41:15 2012
@@ -0,0 +1,40 @@
+PREHOOK: query: -- In this test, the stats publisher does not exists.
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- In this test, the stats publisher does not exists.
+-- If stats need not be reliable, the statement succeeds. However, if stats are supposed
+-- to be reliable (by setting hive.stats.reliable to true), the insert statement fails
+-- because stats cannot be collected for this statement
+
+create table tmptable(key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tmptable
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+Execution failed with exit status: 2
+Obtaining error information
+
+Task failed!
+Task ID:
+  Stage-1
+
+Logs:
+
+#### A masked pattern was here ####
+FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.MapRedTask

Added: hive/trunk/ql/src/test/results/clientpositive/stats_aggregator_error_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/stats_aggregator_error_1.q.out?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/stats_aggregator_error_1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/stats_aggregator_error_1.q.out Thu Jul 26 17:41:15 2012
@@ -0,0 +1,125 @@
+PREHOOK: query: -- In this test, there is a dummy stats aggregator which throws an error when various
+-- methods are called (as indicated by the parameter hive.test.dummystats.agregator)
+-- Since stats need not be reliable (by setting hive.stats.reliable to false), the 
+-- insert statements succeed. The insert statement succeeds even if the stats aggregator
+-- is set to null, since stats need not be reliable.
+
+create table tmptable(key string, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- In this test, there is a dummy stats aggregator which throws an error when various
+-- methods are called (as indicated by the parameter hive.test.dummystats.agregator)
+-- Since stats need not be reliable (by setting hive.stats.reliable to false), the 
+-- insert statements succeed. The insert statement succeeds even if the stats aggregator
+-- is set to null, since stats need not be reliable.
+
+create table tmptable(key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tmptable
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500

Added: hive/trunk/ql/src/test/results/clientpositive/stats_publisher_error_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientpositive/stats_publisher_error_1.q.out?rev=1366103&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientpositive/stats_publisher_error_1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientpositive/stats_publisher_error_1.q.out Thu Jul 26 17:41:15 2012
@@ -0,0 +1,125 @@
+PREHOOK: query: -- In this test, there is a dummy stats publisher which throws an error when various
+-- methods are called (as indicated by the parameter hive.test.dummystats.publisher)
+-- Since stats need not be reliable (by setting hive.stats.reliable to false), the
+-- insert statements succeed. The insert statement succeeds even if the stats publisher
+-- is set to null, since stats need not be reliable.
+
+create table tmptable(key string, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: -- In this test, there is a dummy stats publisher which throws an error when various
+-- methods are called (as indicated by the parameter hive.test.dummystats.publisher)
+-- Since stats need not be reliable (by setting hive.stats.reliable to false), the
+-- insert statements succeed. The insert statement succeeds even if the stats publisher
+-- is set to null, since stats need not be reliable.
+
+create table tmptable(key string, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@tmptable
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500
+PREHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@tmptable
+POSTHOOK: query: INSERT OVERWRITE TABLE tmptable select * from src
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@tmptable
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select count(1) from tmptable
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: query: select count(1) from tmptable
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tmptable
+#### A masked pattern was here ####
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.key SIMPLE [(src)src.FieldSchema(name:key, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: tmptable.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+500