You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2007/05/07 23:33:04 UTC

svn commit: r536000 - in /lucene/hadoop/trunk: ./ src/java/org/apache/hadoop/mapred/

Author: cutting
Date: Mon May  7 14:33:03 2007
New Revision: 536000

URL: http://svn.apache.org/viewvc?view=rev&rev=536000
Log:
HADOOP-1324.  Change so that an FSError kills only the task that generates it rather than the entire task tracker.  Contributed by Arun.

Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/IsolationRunner.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskRunner.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=536000&r1=535999&r2=536000
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Mon May  7 14:33:03 2007
@@ -356,6 +356,10 @@
      More care is also taken to not allocate files on full or offline
      drives.  (Devaraj Das via cutting)
 
+106. HADOOP-1324.  Change so that an FSError kills only the task that
+     generates it rather than the entire task tracker.
+     (Arun C Murthy via cutting)
+
 
 Release 0.12.3 - 2007-04-06
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/IsolationRunner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/IsolationRunner.java?view=diff&rev=536000&r1=535999&r2=536000
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/IsolationRunner.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/IsolationRunner.java Mon May  7 14:33:03 2007
@@ -47,8 +47,8 @@
       LOG.info("Task " + taskid + " reporting done.");
     }
 
-    public void fsError(String message) throws IOException {
-      LOG.info("Task reporting file system error: " + message);
+    public void fsError(String taskId, String message) throws IOException {
+      LOG.info("Task " + taskId + " reporting file system error: " + message);
     }
 
     public Task getTask(String taskid) throws IOException {

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java?view=diff&rev=536000&r1=535999&r2=536000
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java Mon May  7 14:33:03 2007
@@ -236,8 +236,9 @@
       }
     }
 
-    public synchronized void fsError(String message) throws IOException {
-      LOG.fatal("FSError: "+ message);
+    public synchronized void fsError(String taskId, String message) 
+    throws IOException {
+      LOG.fatal("FSError: "+ message + "from task: " + taskId);
     }
 
     public TaskCompletionEvent[] getMapCompletionEvents(

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskRunner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskRunner.java?view=diff&rev=536000&r1=535999&r2=536000
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskRunner.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskRunner.java Mon May  7 14:33:03 2007
@@ -289,7 +289,7 @@
     } catch (FSError e) {
       LOG.fatal("FSError", e);
       try {
-        tracker.fsError(e.getMessage());
+        tracker.fsError(t.getTaskId(), e.getMessage());
       } catch (IOException ie) {
         LOG.fatal(t.getTaskId()+" reporting FSError", ie);
       }

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java?view=diff&rev=536000&r1=535999&r2=536000
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java Mon May  7 14:33:03 2007
@@ -1577,11 +1577,15 @@
     }
   }
 
-  /** A child task had a local filesystem error.  Exit, so that no future
-   * jobs are accepted. */
-  public synchronized void fsError(String message) throws IOException {
-    LOG.fatal("FSError, exiting: "+ message);
-    running = false;
+  /** 
+   * A child task had a local filesystem error. Kill the task.
+   */  
+  public synchronized void fsError(String taskId, String message) 
+  throws IOException {
+    LOG.fatal("Task: " + taskId + " - Killed due to FSError: " + message);
+    TaskInProgress tip = runningTasks.get(taskId);
+    tip.reportDiagnosticInfo("FSError: " + message);
+    purgeTask(tip);
   }
 
   public TaskCompletionEvent[] getMapCompletionEvents(
@@ -1705,7 +1709,7 @@
         task.run(job, umbilical);             // run the task
       } catch (FSError e) {
         LOG.fatal("FSError from child", e);
-        umbilical.fsError(e.getMessage());
+        umbilical.fsError(taskid, e.getMessage());
       } catch (Throwable throwable) {
         LOG.warn("Error running child", throwable);
         // Report back any failures, for diagnostic purposes

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java?view=diff&rev=536000&r1=535999&r2=536000
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java Mon May  7 14:33:03 2007
@@ -63,7 +63,7 @@
   void done(String taskid) throws IOException;
 
   /** Report that the task encounted a local filesystem error.*/
-  void fsError(String message) throws IOException;
+  void fsError(String taskId, String message) throws IOException;
 
   /** Called by a reduce task to get the map output locations for finished maps.
    *