hadoop-common-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From omal...@apache.org
Subject svn commit: r1077705 - /hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/ReduceTask.java
Date Fri, 04 Mar 2011 04:46:28 GMT
Author: omalley
Date: Fri Mar  4 04:46:28 2011
New Revision: 1077705

URL: http://svn.apache.org/viewvc?rev=1077705&view=rev
Log:
commit bd2da1fa328e4c0fec0ba0eabba6fead891b5eb0
Author: Arun C Murthy <acmurthy@apache.org>
Date:   Sun Sep 19 14:59:37 2010 -0700

    Reduce logging during shuffle.

Modified:
    hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/ReduceTask.java

Modified: hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/ReduceTask.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/ReduceTask.java?rev=1077705&r1=1077704&r2=1077705&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/ReduceTask.java
(original)
+++ hadoop/common/branches/branch-0.20-security-patches/src/mapred/org/apache/hadoop/mapred/ReduceTask.java
Fri Mar  4 04:46:28 2011
@@ -1428,8 +1428,10 @@ class ReduceTask extends Task {
               " arrived to reduce task " + reduce);
           return null;
         }
-        LOG.info("header: " + mapId + ", compressed len: " + compressedLength +
-                 ", decompressed len: " + decompressedLength);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("header: " + mapId + ", compressed len: " + compressedLength +
+              ", decompressed len: " + decompressedLength);
+        }
 
         //We will put a file in memory if it meets certain criteria:
         //1. The size of the (decompressed) file should be less than 25% of 
@@ -1441,19 +1443,23 @@ class ReduceTask extends Task {
 
         // Shuffle
         MapOutput mapOutput = null;
-        if (shuffleInMemory) { 
-          LOG.info("Shuffling " + decompressedLength + " bytes (" + 
-              compressedLength + " raw bytes) " + 
-              "into RAM from " + mapOutputLoc.getTaskAttemptId());
+        if (shuffleInMemory) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Shuffling " + decompressedLength + " bytes (" + 
+                compressedLength + " raw bytes) " + 
+                "into RAM from " + mapOutputLoc.getTaskAttemptId());
+          }
 
           mapOutput = shuffleInMemory(mapOutputLoc, connection, input,
                                       (int)decompressedLength,
                                       (int)compressedLength);
         } else {
-          LOG.info("Shuffling " + decompressedLength + " bytes (" + 
-              compressedLength + " raw bytes) " + 
-              "into Local-FS from " + mapOutputLoc.getTaskAttemptId());
-
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Shuffling " + decompressedLength + " bytes (" + 
+                compressedLength + " raw bytes) " + 
+                "into Local-FS from " + mapOutputLoc.getTaskAttemptId());
+          }
+          
           mapOutput = shuffleToDisk(mapOutputLoc, input, filename, 
               compressedLength);
         }
@@ -1602,8 +1608,10 @@ class ReduceTask extends Task {
                            (shuffleData.length-bytesRead));
           }
 
-          LOG.info("Read " + bytesRead + " bytes from map-output for " +
-                   mapOutputLoc.getTaskAttemptId());
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Read " + bytesRead + " bytes from map-output for " +
+                mapOutputLoc.getTaskAttemptId());
+          }
 
           input.close();
         } catch (IOException ioe) {
@@ -1658,13 +1666,15 @@ class ReduceTask extends Task {
         }
 
         // TODO: Remove this after a 'fix' for HADOOP-3647
-        if (mapOutputLength > 0) {
-          DataInputBuffer dib = new DataInputBuffer();
-          dib.reset(shuffleData, 0, shuffleData.length);
-          LOG.info("Rec #1 from " + mapOutputLoc.getTaskAttemptId() + " -> (" + 
-                   WritableUtils.readVInt(dib) + ", " + 
-                   WritableUtils.readVInt(dib) + ") from " + 
-                   mapOutputLoc.getHost());
+        if (LOG.isDebugEnabled()) {
+          if (mapOutputLength > 0) {
+            DataInputBuffer dib = new DataInputBuffer();
+            dib.reset(shuffleData, 0, shuffleData.length);
+            LOG.debug("Rec #1 from " + mapOutputLoc.getTaskAttemptId() + 
+                " -> (" + WritableUtils.readVInt(dib) + ", " + 
+                WritableUtils.readVInt(dib) + ") from " + 
+                mapOutputLoc.getHost());
+          }
         }
         
         return mapOutput;
@@ -2732,9 +2742,11 @@ class ReduceTask extends Task {
         do {
           try {
             int numNewMaps = getMapCompletionEvents();
-            if (numNewMaps > 0) {
-              LOG.info(reduceTask.getTaskID() + ": " +  
-                  "Got " + numNewMaps + " new map-outputs"); 
+            if (LOG.isDebugEnabled()) {
+              if (numNewMaps > 0) {
+                LOG.debug(reduceTask.getTaskID() + ": " +  
+                    "Got " + numNewMaps + " new map-outputs"); 
+              }
             }
             Thread.sleep(SLEEP_TIME);
           } 



Mime
View raw message