Bladeren bron

HADOOP-10005. No need to check INFO severity level is enabled or not. Contributed by Jackie Chang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1532907 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 11 jaren geleden
bovenliggende
commit
dea1b2e84a

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -585,6 +585,9 @@ Release 2.1.1-beta - 2013-09-23
     HADOOP-9977. Hadoop services won't start with different keypass and
     keystorepass when https is enabled. (cnauroth)
 
+    HADOOP-10005. No need to check INFO severity level is enabled or not.
+    (Jackie Chang via suresh)
+
 Release 2.1.0-beta - 2013-08-22
 
   INCOMPATIBLE CHANGES

+ 6 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -1315,15 +1315,13 @@ public class DataNode extends Configured
     
     int numTargets = xferTargets.length;
     if (numTargets > 0) {
-      if (LOG.isInfoEnabled()) {
-        StringBuilder xfersBuilder = new StringBuilder();
-        for (int i = 0; i < numTargets; i++) {
-          xfersBuilder.append(xferTargets[i]);
-          xfersBuilder.append(" ");
-        }
-        LOG.info(bpReg + " Starting thread to transfer " + 
-                 block + " to " + xfersBuilder);                       
+      StringBuilder xfersBuilder = new StringBuilder();
+      for (int i = 0; i < numTargets; i++) {
+        xfersBuilder.append(xferTargets[i]);
+        xfersBuilder.append(" ");
       }
+      LOG.info(bpReg + " Starting thread to transfer " + 
+               block + " to " + xfersBuilder);                       
 
       new Daemon(new DataTransfer(xferTargets, block,
           BlockConstructionStage.PIPELINE_SETUP_CREATE, "")).start();

+ 22 - 32
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java

@@ -972,12 +972,10 @@ public class MapTask extends Task {
       maxRec = kvmeta.capacity() / NMETA;
       softLimit = (int)(kvbuffer.length * spillper);
       bufferRemaining = softLimit;
-      if (LOG.isInfoEnabled()) {
-        LOG.info(JobContext.IO_SORT_MB + ": " + sortmb);
-        LOG.info("soft limit at " + softLimit);
-        LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
-        LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
-      }
+      LOG.info(JobContext.IO_SORT_MB + ": " + sortmb);
+      LOG.info("soft limit at " + softLimit);
+      LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
 
       // k/v serialization
       comparator = job.getOutputKeyComparator();
@@ -1180,10 +1178,8 @@ public class MapTask extends Task {
       final int aligned = pos - (pos % METASIZE);
       kvindex =
         ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
-            "(" + (kvindex * 4) + ")");
-      }
+      LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
+          "(" + (kvindex * 4) + ")");
     }
 
     /**
@@ -1198,10 +1194,8 @@ public class MapTask extends Task {
       // set start/end to point to first meta record
       kvstart = kvend =
         ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
-          (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
-      }
+      LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
+        (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
     }
 
     /**
@@ -1456,15 +1450,13 @@ public class MapTask extends Task {
         if (kvindex != kvend) {
           kvend = (kvindex + NMETA) % kvmeta.capacity();
           bufend = bufmark;
-          if (LOG.isInfoEnabled()) {
-            LOG.info("Spilling map output");
-            LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-                     "; bufvoid = " + bufvoid);
-            LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-                     "); kvend = " + kvend + "(" + (kvend * 4) +
-                     "); length = " + (distanceTo(kvend, kvstart,
-                           kvmeta.capacity()) + 1) + "/" + maxRec);
-          }
+          LOG.info("Spilling map output");
+          LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+                   "; bufvoid = " + bufvoid);
+          LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+                   "); kvend = " + kvend + "(" + (kvend * 4) +
+                   "); length = " + (distanceTo(kvend, kvstart,
+                         kvmeta.capacity()) + 1) + "/" + maxRec);
           sortAndSpill();
         }
       } catch (InterruptedException e) {
@@ -1547,15 +1539,13 @@ public class MapTask extends Task {
       kvend = (kvindex + NMETA) % kvmeta.capacity();
       bufend = bufmark;
       spillInProgress = true;
-      if (LOG.isInfoEnabled()) {
-        LOG.info("Spilling map output");
-        LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-                 "; bufvoid = " + bufvoid);
-        LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-                 "); kvend = " + kvend + "(" + (kvend * 4) +
-                 "); length = " + (distanceTo(kvend, kvstart,
-                       kvmeta.capacity()) + 1) + "/" + maxRec);
-      }
+      LOG.info("Spilling map output");
+      LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+               "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+               "); kvend = " + kvend + "(" + (kvend * 4) +
+               "); length = " + (distanceTo(kvend, kvstart,
+                     kvmeta.capacity()) + 1) + "/" + maxRec);
       spillReady.signal();
     }
 

+ 6 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMultiFileInputFormat.java

@@ -81,14 +81,12 @@ public class TestMultiFileInputFormat extends TestCase{
   }
   
   public void testFormat() throws IOException {
-    if(LOG.isInfoEnabled()) {
-      LOG.info("Test started");
-      LOG.info("Max split count           = " + MAX_SPLIT_COUNT);
-      LOG.info("Split count increment     = " + SPLIT_COUNT_INCR);
-      LOG.info("Max bytes per file        = " + MAX_BYTES);
-      LOG.info("Max number of files       = " + MAX_NUM_FILES);
-      LOG.info("Number of files increment = " + NUM_FILES_INCR);
-    }
+    LOG.info("Test started");
+    LOG.info("Max split count           = " + MAX_SPLIT_COUNT);
+    LOG.info("Split count increment     = " + SPLIT_COUNT_INCR);
+    LOG.info("Max bytes per file        = " + MAX_BYTES);
+    LOG.info("Max number of files       = " + MAX_NUM_FILES);
+    LOG.info("Number of files increment = " + NUM_FILES_INCR);
     
     MultiFileInputFormat<Text,Text> format = new DummyMultiFileInputFormat();
     FileSystem fs = FileSystem.getLocal(job);

+ 3 - 5
hadoop-tools/hadoop-gridmix/src/test/java/org/apache/hadoop/mapred/gridmix/DebugJobProducer.java

@@ -122,11 +122,9 @@ public class DebugJobProducer implements JobStoryProducer {
       // Add/remove excess
       recs[0] += totalrecs - tot_recs;
       bytes[0] += totalbytes - tot_bytes;
-      if (LOG.isInfoEnabled()) {
-        LOG.info(
-          "DIST: " + Arrays.toString(recs) + " " + tot_recs + "/" + totalrecs +
-            " " + Arrays.toString(bytes) + " " + tot_bytes + "/" + totalbytes);
-      }
+      LOG.info(
+        "DIST: " + Arrays.toString(recs) + " " + tot_recs + "/" + totalrecs +
+          " " + Arrays.toString(bytes) + " " + tot_bytes + "/" + totalbytes);
     }
 
     private static final AtomicInteger seq = new AtomicInteger(0);