Browse Source

HADOOP-3517. Fixes a problem in the reducer due to which the last InMemory merge may be missed. Contributed by Arun Murthy.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@667040 13f79535-47bb-0310-9956-ffa450edef68
Devaraj Das 17 years ago
parent
commit
f8495b9f9b

+ 3 - 0
CHANGES.txt

@@ -588,6 +588,9 @@ Release 0.18.0 - Unreleased
     HADOOP-3544. Fixes a documentation issue for hadoop archives.
     (Mahadev Konar via ddas)
 
+    HADOOP-3517. Fixes a problem in the reducer due to which the last InMemory
+    merge may be missed. (Arun Murthy via ddas)
+
 Release 0.17.0 - 2008-05-18
 
   INCOMPATIBLE CHANGES

+ 4 - 1
docs/changes.html

@@ -313,7 +313,7 @@ InputFormat.validateInput.<br />(tomwhite via omalley)</li>
     </ol>
   </li>
   <li><a href="javascript:toggleList('release_0.18.0_-_unreleased_._bug_fixes_')">  BUG FIXES
-</a>&nbsp;&nbsp;&nbsp;(85)
+</a>&nbsp;&nbsp;&nbsp;(86)
     <ol id="release_0.18.0_-_unreleased_._bug_fixes_">
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-2905">HADOOP-2905</a>. 'fsck -move' triggers NPE in NameNode.<br />(Lohit Vjayarenu via rangadi)</li>
       <li>Increment ClientProtocol.versionID missed by <a href="http://issues.apache.org/jira/browse/HADOOP-2585">HADOOP-2585</a>.<br />(shv)</li>
@@ -488,6 +488,9 @@ in the subdirectory are removed. ((Tsz Wo (Nicholas), SZE via dhruba)
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3542">HADOOP-3542</a>. Diables the creation of _logs directory for the archives
 directory.<br />(Mahadev Konar via ddas)</li>
       <li><a href="http://issues.apache.org/jira/browse/HADOOP-3544">HADOOP-3544</a>. Fixes a documentation issue for hadoop archives.<br />(Mahadev Konar via ddas)</li>
+      <li><a href="http://issues.apache.org/jira/browse/HADOOP-3517">HADOOP-3517</a>. Fixes a problem in the reducer due to which the last InMemory
+merge may be missed.
+</li>
     </ol>
   </li>
 </ul>

+ 9 - 21
docs/mapred_tutorial.html

@@ -301,7 +301,7 @@ document.write("Last Published: " + document.lastModified);
 <a href="#Example%3A+WordCount+v2.0">Example: WordCount v2.0</a>
 <ul class="minitoc">
 <li>
-<a href="#Source+Code-N10D77">Source Code</a>
+<a href="#Source+Code-N10D60">Source Code</a>
 </li>
 <li>
 <a href="#Sample+Runs">Sample Runs</a>
@@ -2172,20 +2172,8 @@ document.write("Last Published: " + document.lastModified);
             JobConf.setCompressMapOutput(boolean)</a> api and the 
             <span class="codefrag">CompressionCodec</span> to be used via the
             <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressorClass(java.lang.Class)">
-            JobConf.setMapOutputCompressorClass(Class)</a> api. Since 
-            the intermediate map-outputs are always stored in the 
-            <a href="api/org/apache/hadoop/io/SequenceFile.html">SequenceFile</a>
-            format, the
-            <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html">
-            SequenceFile.CompressionType</a> (i.e.
-            <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html#RECORD">
-            RECORD</a> / 
-            <a href="api/org/apache/hadoop/io/SequenceFile.CompressionType.html#BLOCK">
-            BLOCK</a> - defaults to <span class="codefrag">RECORD</span>) can be specified via the
-            <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressionType(org.apache.hadoop.io.SequenceFile.CompressionType)">
-            JobConf.setMapOutputCompressionType(SequenceFile.CompressionType)</a> 
-            api.</p>
-<a name="N10D2E"></a><a name="Job+Outputs"></a>
+            JobConf.setMapOutputCompressorClass(Class)</a> api.</p>
+<a name="N10D17"></a><a name="Job+Outputs"></a>
 <h5>Job Outputs</h5>
 <p>Applications can control compression of job-outputs via the
             <a href="api/org/apache/hadoop/mapred/OutputFormatBase.html#setCompressOutput(org.apache.hadoop.mapred.JobConf,%20boolean)">
@@ -2197,15 +2185,15 @@ document.write("Last Published: " + document.lastModified);
             <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html">
             SequenceFileOutputFormat</a>, the required
             <span class="codefrag">SequenceFile.CompressionType</span> (i.e. <span class="codefrag">RECORD</span> / 
-            <span class="codefrag">BLOCK</span> - defaults to <span class="codefrag">RECORD</span>)can be specified 
-            via the 
+            <span class="codefrag">BLOCK</span> - defaults to <span class="codefrag">RECORD</span>) can be 
+            specified via the 
             <a href="api/org/apache/hadoop/mapred/SequenceFileOutputFormat.html#setOutputCompressionType(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.io.SequenceFile.CompressionType)">
             SequenceFileOutputFormat.setOutputCompressionType(JobConf, 
             SequenceFile.CompressionType)</a> api.</p>
 </div>
 
     
-<a name="N10D5D"></a><a name="Example%3A+WordCount+v2.0"></a>
+<a name="N10D46"></a><a name="Example%3A+WordCount+v2.0"></a>
 <h2 class="h3">Example: WordCount v2.0</h2>
 <div class="section">
 <p>Here is a more complete <span class="codefrag">WordCount</span> which uses many of the
@@ -2215,7 +2203,7 @@ document.write("Last Published: " + document.lastModified);
       <a href="quickstart.html#SingleNodeSetup">pseudo-distributed</a> or
       <a href="quickstart.html#Fully-Distributed+Operation">fully-distributed</a> 
       Hadoop installation.</p>
-<a name="N10D77"></a><a name="Source+Code-N10D77"></a>
+<a name="N10D60"></a><a name="Source+Code-N10D60"></a>
 <h3 class="h4">Source Code</h3>
 <table class="ForrestTable" cellspacing="1" cellpadding="4">
           
@@ -3425,7 +3413,7 @@ document.write("Last Published: " + document.lastModified);
 </tr>
         
 </table>
-<a name="N114D9"></a><a name="Sample+Runs"></a>
+<a name="N114C2"></a><a name="Sample+Runs"></a>
 <h3 class="h4">Sample Runs</h3>
 <p>Sample text-files as input:</p>
 <p>
@@ -3593,7 +3581,7 @@ document.write("Last Published: " + document.lastModified);
 <br>
         
 </p>
-<a name="N115AD"></a><a name="Highlights"></a>
+<a name="N11596"></a><a name="Highlights"></a>
 <h3 class="h4">Highlights</h3>
 <p>The second version of <span class="codefrag">WordCount</span> improves upon the 
         previous one by using some features offered by the Map-Reduce framework:

File diff suppressed because it is too large
+ 1 - 1
docs/mapred_tutorial.pdf


+ 2 - 2
src/core/org/apache/hadoop/io/compress/CodecPool.java

@@ -98,7 +98,7 @@ public class CodecPool {
     Compressor compressor = borrow(compressorPool, codec.getCompressorType());
     if (compressor == null) {
       compressor = codec.createCompressor();
-      LOG.debug("Got brand-new compressor");
+      LOG.info("Got brand-new compressor");
     } else {
       LOG.debug("Got recycled compressor");
     }
@@ -118,7 +118,7 @@ public class CodecPool {
     Decompressor decompressor = borrow(decompressorPool, codec.getDecompressorType());
     if (decompressor == null) {
       decompressor = codec.createDecompressor();
-      LOG.debug("Got brand-new decompressor");
+      LOG.info("Got brand-new decompressor");
     } else {
       LOG.debug("Got recycled decompressor");
     }

+ 3 - 15
src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -1665,19 +1665,7 @@
             JobConf.setCompressMapOutput(boolean)</a> api and the 
             <code>CompressionCodec</code> to be used via the
             <a href="ext:api/org/apache/hadoop/mapred/jobconf/setmapoutputcompressorclass">
-            JobConf.setMapOutputCompressorClass(Class)</a> api. Since 
-            the intermediate map-outputs are always stored in the 
-            <a href="ext:api/org/apache/hadoop/io/sequencefile">SequenceFile</a>
-            format, the
-            <a href="ext:api/org/apache/hadoop/io/sequencefilecompressiontype">
-            SequenceFile.CompressionType</a> (i.e.
-            <a href="ext:api/org/apache/hadoop/io/sequencefilecompressiontype/record">
-            RECORD</a> / 
-            <a href="ext:api/org/apache/hadoop/io/sequencefilecompressiontype/block">
-            BLOCK</a> - defaults to <code>RECORD</code>) can be specified via the
-            <a href="ext:api/org/apache/hadoop/mapred/jobconf/setmapoutputcompressiontype">
-            JobConf.setMapOutputCompressionType(SequenceFile.CompressionType)</a> 
-            api.</p>
+            JobConf.setMapOutputCompressorClass(Class)</a> api.</p>
           </section>
           
           <section>
@@ -1694,8 +1682,8 @@
             <a href="ext:api/org/apache/hadoop/mapred/sequencefileoutputformat">
             SequenceFileOutputFormat</a>, the required
             <code>SequenceFile.CompressionType</code> (i.e. <code>RECORD</code> / 
-            <code>BLOCK</code> - defaults to <code>RECORD</code>)can be specified 
-            via the 
+            <code>BLOCK</code> - defaults to <code>RECORD</code>) can be 
+            specified via the 
             <a href="ext:api/org/apache/hadoop/mapred/sequencefileoutputformat/setoutputcompressiontype">
             SequenceFileOutputFormat.setOutputCompressionType(JobConf, 
             SequenceFile.CompressionType)</a> api.</p>

+ 0 - 1
src/docs/src/documentation/content/xdocs/site.xml

@@ -164,7 +164,6 @@ See http://forrest.apache.org/docs/linking.html for more info.
                 <setmaxreducetaskfailurespercent href="#setMaxReduceTaskFailuresPercent(int)" />
                 <setjobendnotificationuri href="#setJobEndNotificationURI(java.lang.String)" />
                 <setcompressmapoutput href="#setCompressMapOutput(boolean)" />
-                <setmapoutputcompressiontype href="#setMapOutputCompressionType(org.apache.hadoop.io.SequenceFile.CompressionType)" />
                 <setmapoutputcompressorclass href="#setMapOutputCompressorClass(java.lang.Class)" />
                 <getjoblocaldir href="#getJobLocalDir()" />
                 <getjar href="#getJar()" />

+ 6 - 3
src/mapred/org/apache/hadoop/mapred/Merger.java

@@ -43,7 +43,11 @@ class Merger {
   private static final Log LOG = LogFactory.getLog(Merger.class);
   
   private static final long PROGRESS_BAR = 10000;
-  
+
+  // Local directories
+  private static LocalDirAllocator lDirAlloc = 
+    new LocalDirAllocator("mapred.local.dir");
+
   public static <K extends Object, V extends Object>
   RawKeyValueIterator merge(Configuration conf, FileSystem fs,
                             Class<K> keyClass, Class<V> valueClass, 
@@ -275,7 +279,6 @@ class Merger {
       int numSegments = segments.size();
       int origFactor = factor;
       int passNo = 1;
-      LocalDirAllocator lDirAlloc = new LocalDirAllocator("mapred.local.dir");
       do {
         //get the factor for this pass of merge
         factor = getPassFactor(factor, passNo, numSegments);
@@ -413,4 +416,4 @@ class Merger {
     }
 
   }
-}
+}

+ 3 - 1
src/mapred/org/apache/hadoop/mapred/RamManager.java

@@ -28,10 +28,12 @@ interface RamManager {
    * 
    * @param requestedSize size of memory requested
    * @param in input stream
+   * @throws InterruptedException
    * @return <code>true</code> if memory was allocated immediately, 
    *         else <code>false</code>
    */
-  boolean reserve(int requestedSize, InputStream in);
+  boolean reserve(int requestedSize, InputStream in) 
+  throws InterruptedException;
   
   /**
    * Return memory to the pool.

+ 98 - 57
src/mapred/org/apache/hadoop/mapred/ReduceTask.java

@@ -422,11 +422,6 @@ class ReduceTask extends Task {
      */
     private volatile boolean exitLocalFSMerge = false;
     
-    /** 
-     * A flag to indicate when to exit InMemMerge
-     */
-    private volatile boolean exitInMemMerge = false;
-    
     /**
      * When we accumulate mergeThreshold number of files in ram, we merge/spill
      */
@@ -712,42 +707,60 @@ class ReduceTask extends Task {
     class ShuffleRamManager implements RamManager {
       /* Maximum percentage of the in-memory limit that a single shuffle can 
        * consume*/ 
-      private static final float MAX_SINGLE_SHUFFLE_SEGMENT_PERCENT = 0.25f;
+      private static final float MAX_SINGLE_SHUFFLE_SEGMENT_FRACTION = 0.25f;
       
-      private boolean closed = false;
+      /* Maximum percentage of shuffle-threads which can be stalled 
+       * simultaneously after which a merge is triggered. */ 
+      private static final float MAX_STALLED_SHUFFLE_THREADS_FRACTION = 0.75f;
       
-      volatile private int numClosed = 0;
-      volatile private int size = 0;
       private final int maxSize;
       private final int maxSingleShuffleLimit;
       
+      private int size = 0;
+      
       private Object dataAvailable = new Object();
-      private volatile int fullSize = 0;
+      private int fullSize = 0;
+      private int numPendingRequests = 0;
+      private int numRequiredMapOutputs = 0;
+      private int numClosed = 0;
+      private boolean closed = false;
       
       public ShuffleRamManager(Configuration conf) {
         maxSize = conf.getInt("fs.inmemory.size.mb", 100) * 1024 * 1024;
-        maxSingleShuffleLimit = (int)(maxSize * MAX_SINGLE_SHUFFLE_SEGMENT_PERCENT);
+        maxSingleShuffleLimit = (int)(maxSize * MAX_SINGLE_SHUFFLE_SEGMENT_FRACTION);
         LOG.info("ShuffleRamManager: MemoryLimit=" + maxSize + 
                  ", MaxSingleShuffleLimit=" + maxSingleShuffleLimit);
       }
       
-      public synchronized boolean reserve(int requestedSize, InputStream in) {
+      public synchronized boolean reserve(int requestedSize, InputStream in) 
+      throws InterruptedException {
+        // Wait till the request can be fulfilled...
         while ((size + requestedSize) > maxSize) {
-          try {
-            // Close the connection
-            if (in != null) {
-              try {
-                in.close();
-              } catch (IOException ie) {
-                LOG.info("Failed to close connection with: " + ie);
-              } finally {
-                in = null;
-              }
+          
+          // Close the input...
+          if (in != null) {
+            try {
+              in.close();
+            } catch (IOException ie) {
+              LOG.info("Failed to close connection with: " + ie);
+            } finally {
+              in = null;
             }
-            
-            // Wait for memory to free up
-            wait();
-          } catch (InterruptedException ie) {}
+          } 
+
+          // Track pending requests
+          synchronized (dataAvailable) {
+            ++numPendingRequests;
+            dataAvailable.notify();
+          }
+
+          // Wait for memory to free up
+          wait();
+          
+          // Track pending requests
+          synchronized (dataAvailable) {
+            --numPendingRequests;
+          }
         }
         
         size += requestedSize;
@@ -767,20 +780,25 @@ class ReduceTask extends Task {
         notifyAll();
       }
       
-      public void waitForDataToMerge() {
+      public boolean waitForDataToMerge() throws InterruptedException {
+        boolean done = false;
         synchronized (dataAvailable) {
           while (!closed &&
                  (getPercentUsed() < MAX_INMEM_FILESYS_USE ||
-                  getReservedFiles() < 
+                  numClosed < 
                     (int)(MAX_INMEM_FILESYS_USE/MAX_INMEM_FILESIZE_FRACTION)
                  ) 
                  &&
-                 (mergeThreshold <= 0 || getReservedFiles() < mergeThreshold)) {
-            try {
-              dataAvailable.wait();
-            } catch (InterruptedException ie) {}
+                 (mergeThreshold <= 0 || numClosed < mergeThreshold) 
+                 && 
+                 (numPendingRequests < 
+                      numCopiers*MAX_STALLED_SHUFFLE_THREADS_FRACTION && 
+                   numPendingRequests < numRequiredMapOutputs)) {
+            dataAvailable.wait();
           }
+          done = closed;
         }
+        return done;
       }
       
       public void closeInMemoryFile(int requestedSize) {
@@ -791,6 +809,13 @@ class ReduceTask extends Task {
         }
       }
       
+      public void setNumCopiedMapOutputs(int numRequiredMapOutputs) {
+        synchronized (dataAvailable) {
+          this.numRequiredMapOutputs = numRequiredMapOutputs;
+          dataAvailable.notify();
+        }
+      }
+      
       public void close() {
         synchronized (dataAvailable) {
           closed = true;
@@ -799,14 +824,10 @@ class ReduceTask extends Task {
         }
       }
       
-      float getPercentUsed() {
+      private float getPercentUsed() {
         return (float)fullSize/maxSize;
       }
-      
-      int getReservedFiles() {
-        return numClosed;
-      }
-      
+
       int getMemoryLimit() {
         return maxSize;
       }
@@ -978,7 +999,8 @@ class ReduceTask extends Task {
             }
             
             // Note that we successfully copied the map-output
-            copiedMapOutputs.add(loc.getTaskId());
+            noteCopiedMapOutput(loc.getTaskId());
+            
             return bytes;
           }
           
@@ -1004,12 +1026,22 @@ class ReduceTask extends Task {
           }
 
           // Note that we successfully copied the map-output
-          copiedMapOutputs.add(loc.getTaskId());
+          noteCopiedMapOutput(loc.getTaskId());
         }
         
         return bytes;
       }
       
+      /**
+       * Save the map taskid whose output we just copied.
+       * This function assumes that it has been synchronized on ReduceTask.this.
+       * 
+       * @param taskId map taskid
+       */
+      private void noteCopiedMapOutput(TaskID taskId) {
+        copiedMapOutputs.add(taskId);
+        ramManager.setNumCopiedMapOutputs(numMaps - copiedMapOutputs.size());
+      }
 
       /**
        * Get the map output into a local file (either in the inmemory fs or on the 
@@ -1248,6 +1280,7 @@ class ReduceTask extends Task {
       this.shuffleClientMetrics = new ShuffleClientMetrics(conf);
       this.umbilical = umbilical;      
       this.reduceTask = ReduceTask.this;
+
       this.scheduledCopies = new ArrayList<MapOutputLocation>(100);
       this.copyResults = new ArrayList<CopyResult>(100);    
       this.numCopiers = conf.getInt("mapred.reduce.parallel.copies", 5);
@@ -1304,7 +1337,6 @@ class ReduceTask extends Task {
     
     @SuppressWarnings("unchecked")
     public boolean fetchOutputs() throws IOException {
-      final int      numOutputs = reduceTask.getNumMaps();
       List<MapOutputLocation> knownOutputs = 
         new ArrayList<MapOutputLocation>(numCopiers);
       int totalFailures = 0;
@@ -1316,7 +1348,7 @@ class ReduceTask extends Task {
       LocalFSMerger localFSMergerThread = null;
       InMemFSMergeThread inMemFSMergeThread = null;
       
-      for (int i = 0; i < numOutputs; i++) {
+      for (int i = 0; i < numMaps; i++) {
         copyPhase.addPhase();       // add sub-phase per file
       }
       
@@ -1346,7 +1378,7 @@ class ReduceTask extends Task {
       IntWritable fromEventId = new IntWritable(0);
       
         // loop until we get all required outputs
-        while (copiedMapOutputs.size() < numOutputs && mergeThrowable == null) {
+        while (copiedMapOutputs.size() < numMaps && mergeThrowable == null) {
           
           currentTime = System.currentTimeMillis();
           boolean logNow = false;
@@ -1356,7 +1388,7 @@ class ReduceTask extends Task {
           }
           if (logNow) {
             LOG.info(reduceTask.getTaskID() + " Need another " 
-                   + (numOutputs - copiedMapOutputs.size()) + " map output(s) "
+                   + (numMaps - copiedMapOutputs.size()) + " map output(s) "
                    + "where " + numInFlight + " is already in progress");
           }
           
@@ -1503,7 +1535,7 @@ class ReduceTask extends Task {
               float transferRate = mbs/secsSinceStart;
                 
               copyPhase.startNextPhase();
-              copyPhase.setStatus("copy (" + numCopied + " of " + numOutputs 
+              copyPhase.setStatus("copy (" + numCopied + " of " + numMaps 
                                   + " at " +
                                   mbpsFormat.format(transferRate) +  " MB/s)");
                 
@@ -1640,7 +1672,6 @@ class ReduceTask extends Task {
           mapOutputFilesOnDisk.notify();
         }
         
-        exitInMemMerge = true;
         ramManager.close();
         
         //Do a merge of in-memory files (if there are any)
@@ -1648,9 +1679,13 @@ class ReduceTask extends Task {
           try {
             // Wait for the on-disk merge to complete
             localFSMergerThread.join();
+            LOG.info("Interleaved on-disk merge complete: " + 
+                     mapOutputFilesOnDisk.size() + " files left.");
             
             //wait for an ongoing merge (if it is in flight) to complete
             inMemFSMergeThread.join();
+            LOG.info("In-memory merge complete: " + 
+                     mapOutputsFilesInMemory.size() + " files left.");
             } catch (Throwable t) {
             LOG.warn(reduceTask.getTaskID() +
                      " Final merge of the inmemory files threw an exception: " + 
@@ -1662,7 +1697,7 @@ class ReduceTask extends Task {
             return false;
           }
         }
-        return mergeThrowable == null && copiedMapOutputs.size() == numOutputs;
+        return mergeThrowable == null && copiedMapOutputs.size() == numMaps;
     }
     
     private List<Segment<K, V>> createInMemorySegments() {
@@ -1908,10 +1943,11 @@ class ReduceTask extends Task {
       public void run() {
         LOG.info(reduceTask.getTaskID() + " Thread started: " + getName());
         try {
-          while (!exitInMemMerge) {
-            ramManager.waitForDataToMerge();
+          boolean exit = false;
+          do {
+            exit = ramManager.waitForDataToMerge();
             doInMemMerge();
-          }
+          } while (!exit);
         } catch (Throwable t) {
           LOG.warn(reduceTask.getTaskID() +
                    " Merge of the inmemory files threw an exception: "
@@ -1923,7 +1959,6 @@ class ReduceTask extends Task {
       @SuppressWarnings("unchecked")
       private void doInMemMerge() throws IOException{
         if (mapOutputsFilesInMemory.size() == 0) {
-          LOG.info("Noting to merge... ");
           return;
         }
         
@@ -1953,12 +1988,16 @@ class ReduceTask extends Task {
         RawKeyValueIterator rIter = null;
         final Reporter reporter = getReporter(umbilical);
         try {
+          LOG.info("Initiating in-memory merge with " + noInMemorySegments + 
+                   " segments...");
+          
           rIter = Merger.merge(conf, localFileSys,
                                (Class<K>)conf.getMapOutputKeyClass(),
                                (Class<V>)conf.getMapOutputValueClass(),
                                inMemorySegments, inMemorySegments.size(),
                                new Path(reduceTask.getTaskID().toString()),
                                conf.getOutputKeyComparator(), reporter);
+          
           if (null == combinerClass) {
             Merger.writeFile(rIter, writer, reporter);
           } else {
@@ -1966,6 +2005,12 @@ class ReduceTask extends Task {
             combineAndSpill(rIter, reduceCombineInputCounter);
           }
           writer.close();
+
+          LOG.info(reduceTask.getTaskID() + 
+              " Merge of the " + noInMemorySegments +
+              " files in-memory complete." +
+              " Local file is " + outputPath + " of size " + 
+              localFileSys.getFileStatus(outputPath).getLen());
         } catch (Exception e) { 
           //make sure that we delete the ondisk file that we created 
           //earlier when we invoked cloneFileAttributes
@@ -1973,12 +2018,8 @@ class ReduceTask extends Task {
           throw (IOException)new IOException
                   ("Intermedate merge failed").initCause(e);
         }
-        LOG.info(reduceTask.getTaskID() + 
-                 " Merge of the " + noInMemorySegments +
-                 " files in-memory complete." +
-                 " Local file is " + outputPath + " of size " + 
-                 localFileSys.getFileStatus(outputPath).getLen());
-        
+
+        // Note the output of the merge
         FileStatus status = localFileSys.getFileStatus(outputPath);
         synchronized (mapOutputFilesOnDisk) {
           addToMapOutputFilesOnDisk(status);

Some files were not shown because too many files changed in this diff