You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2014/04/01 00:35:44 UTC

svn commit: r1583478 - in /hive/trunk: cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java

Author: hashutosh
Date: Mon Mar 31 22:35:44 2014
New Revision: 1583478

URL: http://svn.apache.org/r1583478
Log:
HIVE-6779 : Hive cli may get into inconsistent state when Ctrl-C is hit on hadoop2 (Ashutosh Chauhan via Jason Dere)

Modified:
    hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java

Modified: hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1583478&r1=1583477&r2=1583478&view=diff
==============================================================================
--- hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Mon Mar 31 22:35:44 2014
@@ -401,7 +401,6 @@ public class CliDriver {
           // First, kill any running MR jobs
           HadoopJobExecHelper.killRunningJobs();
           HiveInterruptUtils.interrupt();
-          this.cliThread.interrupt();
         }
       });
     }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1583478&r1=1583477&r2=1583478&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Mon Mar 31 22:35:44 2014
@@ -2132,14 +2132,14 @@ public final class Utilities {
    * @return the summary of all the input paths.
    * @throws IOException
    */
-  public static ContentSummary getInputSummary(Context ctx, MapWork work, PathFilter filter)
+  public static ContentSummary getInputSummary(final Context ctx, MapWork work, PathFilter filter)
       throws IOException {
     PerfLogger perfLogger = PerfLogger.getPerfLogger();
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.INPUT_SUMMARY);
 
     long[] summary = {0, 0, 0};
 
-    List<String> pathNeedProcess = new ArrayList<String>();
+    final List<String> pathNeedProcess = new ArrayList<String>();
 
     // Since multiple threads could call this method concurrently, locking
     // this method will avoid number of threads out of control.
@@ -2182,6 +2182,13 @@ public final class Utilities {
       HiveInterruptCallback interrup = HiveInterruptUtils.add(new HiveInterruptCallback() {
         @Override
         public void interrupt() {
+          for (String path : pathNeedProcess) {
+            try {
+              new Path(path).getFileSystem(ctx.getConf()).close();
+            } catch (IOException ignore) {
+                LOG.debug(ignore);
+            }
+          }
           if (executor != null) {
             executor.shutdownNow();
           }