hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From acmur...@apache.org
Subject svn commit: r593748 - in /lucene/hadoop/trunk: CHANGES.txt src/java/org/apache/hadoop/util/CopyFiles.java
Date Sat, 10 Nov 2007 07:51:53 GMT
Author: acmurthy
Date: Fri Nov  9 23:51:50 2007
New Revision: 593748

URL: http://svn.apache.org/viewvc?rev=593748&view=rev
Log:
HADOOP-2174.  Removed the unnecessary Reporter.setStatus call from FSCopyFilesMapper.close
which led to a NPE since the reporter isn't valid in the close method. Contributed by Chris
Douglas.

Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=593748&r1=593747&r2=593748&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Fri Nov  9 23:51:50 2007
@@ -99,6 +99,10 @@
     correct value of 'libhdfs.so', currently it is set to the absolute path of
     libhdfs.so. (acmurthy) 
 
+    HADOOP-2174.  Removed the unnecessary Reporter.setStatus call from
+    FSCopyFilesMapper.close which led to a NPE since the reporter isn't valid
+    in the close method. (Chris Douglas via acmurthy) 
+
 Release 0.15.0 - 2007-11-2
 
   INCOMPATIBLE CHANGES

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java?rev=593748&r1=593747&r2=593748&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/util/CopyFiles.java Fri Nov  9 23:51:50
2007
@@ -235,11 +235,8 @@
     private int skipcount = 0;
     private int copycount = 0;
 
-    // hack
-    private Reporter rep;
-
-    private void updateStatus() {
-      rep.setStatus("Copied: " + copycount + " Skipped: " + skipcount +
+    private void updateStatus(Reporter reporter) {
+      reporter.setStatus("Copied: " + copycount + " Skipped: " + skipcount +
                     " Failed: " + failcount);
     }
 
@@ -290,7 +287,7 @@
           outc.collect(null, new Text("SKIP: " + srcstat.getPath()));
           ++skipcount;
           reporter.incrCounter(Counter.SKIP, 1);
-          updateStatus();
+          updateStatus(reporter);
           return;
         }
         // open src file
@@ -330,7 +327,7 @@
       ++copycount;
       reporter.incrCounter(Counter.BYTESCOPIED, cbcopied);
       reporter.incrCounter(Counter.COPY, 1);
-      updateStatus();
+      updateStatus(reporter);
     }
 
     /** Mapper configuration.
@@ -368,12 +365,11 @@
       FileStatus srcstat = value.input;
       Path dstpath = value.output;
       try {
-        rep = reporter;
         copy(srcstat, dstpath, out, reporter);
       } catch (IOException e) {
         ++failcount;
         reporter.incrCounter(Counter.FAIL, 1);
-        updateStatus();
+        updateStatus(reporter);
         final String sfailure = "FAIL " + dstpath + " : " +
                           StringUtils.stringifyException(e);
         out.collect(null, new Text(sfailure));
@@ -388,18 +384,18 @@
               LOG.debug("Ignoring cleanup exception", ex);
             }
             // update status, so we don't get timed out
-            updateStatus();
+            updateStatus(reporter);
             Thread.sleep(3 * 1000);
           }
         } catch (InterruptedException inte) {
           throw (IOException)new IOException().initCause(inte);
         }
-        updateStatus();
+      } finally {
+        updateStatus(reporter);
       }
     }
 
     public void close() throws IOException {
-      updateStatus();
       if (0 == failcount || ignoreReadFailures) {
         return;
       }



Mime
View raw message