hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hashut...@apache.org
Subject svn commit: r1548371 - /hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
Date Fri, 06 Dec 2013 02:01:59 GMT
Author: hashutosh
Date: Fri Dec  6 02:01:59 2013
New Revision: 1548371

URL: http://svn.apache.org/r1548371
Log:
HIVE-5935 : hive.query.string is not provided to FetchTask (Navis via Ashutosh Chauhan)

Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java?rev=1548371&r1=1548370&r2=1548371&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/Driver.java Fri Dec  6 02:01:59 2013
@@ -20,8 +20,6 @@
 package org.apache.hadoop.hive.ql;
 
 import java.io.DataInput;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
@@ -40,7 +38,6 @@ import org.apache.commons.lang.StringUti
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
@@ -455,27 +452,14 @@ public class Driver implements CommandPr
 
       plan = new QueryPlan(command, sem, perfLogger.getStartTime(PerfLogger.DRIVER_RUN));
 
-      // test Only - serialize the query plan and deserialize it
-      if ("true".equalsIgnoreCase(System.getProperty("test.serialize.qplan"))) {
+      String queryId = plan.getQueryId();
+      String queryStr = plan.getQueryStr();
 
-        String queryPlanFileName = ctx.getLocalScratchDir(true) + Path.SEPARATOR_CHAR
-            + "queryplan.xml";
-        LOG.info("query plan = " + queryPlanFileName);
-        queryPlanFileName = new Path(queryPlanFileName).toUri().getPath();
-
-        // serialize the queryPlan
-        FileOutputStream fos = new FileOutputStream(queryPlanFileName);
-        Utilities.serializePlan(plan, fos, conf);
-        fos.close();
-
-        // deserialize the queryPlan
-        FileInputStream fis = new FileInputStream(queryPlanFileName);
-        QueryPlan newPlan = Utilities.deserializePlan(fis, QueryPlan.class, conf);
-        fis.close();
+      conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
+      conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, queryStr);
 
-        // Use the deserialized plan
-        plan = newPlan;
-      }
+      conf.set("mapreduce.workflow.id", "hive_" + queryId);
+      conf.set("mapreduce.workflow.name", queryStr);
 
       // initialize FetchTask right here
       if (plan.getFetchTask() != null) {
@@ -1161,12 +1145,6 @@ public class Driver implements CommandPr
     String queryId = plan.getQueryId();
     String queryStr = plan.getQueryStr();
 
-    conf.setVar(HiveConf.ConfVars.HIVEQUERYID, queryId);
-    conf.setVar(HiveConf.ConfVars.HIVEQUERYSTRING, queryStr);
-
-    conf.set("mapreduce.workflow.id", "hive_"+queryId);
-    conf.set("mapreduce.workflow.name", queryStr);
-
     maxthreads = HiveConf.getIntVar(conf, HiveConf.ConfVars.EXECPARALLETHREADNUMBER);
 
     try {



Mime
View raw message