Browse Source

HDFS-7978. Add LOG.isDebugEnabled() guard for some LOG.debug(..). Contributed by Walter Su.

(cherry picked from commit ed72daa5df97669906234e8ac9a406d78136b206)
Andrew Wang 10 years ago
parent
commit
b17d8a53fa

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

@@ -58,6 +58,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8009. Signal congestion on the DataNode. (wheat9)
 
+    HDFS-7978. Add LOG.isDebugEnabled() guard for some LOG.debug(..).
+    (Walter Su via wang)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 11 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -369,9 +369,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       return null;
     }
     if (clientContext.getDisableLegacyBlockReaderLocal()) {
-      PerformanceAdvisory.LOG.debug(this + ": can't construct " +
-          "BlockReaderLocalLegacy because " +
-          "disableLegacyBlockReaderLocal is set.");
+        PerformanceAdvisory.LOG.debug("{}: can't construct " +
+            "BlockReaderLocalLegacy because " +
+            "disableLegacyBlockReaderLocal is set.", this);
       return null;
     }
     IOException ioe = null;
@@ -410,8 +410,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
                       getPathInfo(inetSocketAddress, conf);
     }
     if (!pathInfo.getPathState().getUsableForShortCircuit()) {
-      PerformanceAdvisory.LOG.debug(this + ": " + pathInfo + " is not " +
-          "usable for short circuit; giving up on BlockReaderLocal.");
+      PerformanceAdvisory.LOG.debug("{}: {} is not usable for short circuit; " +
+              "giving up on BlockReaderLocal.", this, pathInfo);
       return null;
     }
     ShortCircuitCache cache = clientContext.getShortCircuitCache();
@@ -426,11 +426,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       throw exc;
     }
     if (info.getReplica() == null) {
-      if (LOG.isTraceEnabled()) {
-        PerformanceAdvisory.LOG.debug(this + ": failed to get " +
-            "ShortCircuitReplica. Cannot construct " +
-            "BlockReaderLocal via " + pathInfo.getPath());
-      }
+      PerformanceAdvisory.LOG.debug("{}: failed to get " +
+          "ShortCircuitReplica. Cannot construct " +
+          "BlockReaderLocal via {}", this, pathInfo.getPath());
       return null;
     }
     return new BlockReaderLocal.Builder(conf).
@@ -610,9 +608,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
                       getPathInfo(inetSocketAddress, conf);
     }
     if (!pathInfo.getPathState().getUsableForDataTransfer()) {
-      PerformanceAdvisory.LOG.debug(this + ": not trying to create a " +
-          "remote block reader because the UNIX domain socket at " +
-          pathInfo + " is not usable.");
+      PerformanceAdvisory.LOG.debug("{}: not trying to create a " +
+          "remote block reader because the UNIX domain socket at {}" +
+           " is not usable.", this, pathInfo);
       return null;
     }
     if (LOG.isTraceEnabled()) {

+ 8 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java

@@ -319,12 +319,16 @@ public class HAUtil {
             buildTokenServicePrefixForLogicalUri(HdfsConstants.HDFS_URI_SCHEME)
                 + "//" + specificToken.getService());
         ugi.addToken(alias, specificToken);
-        LOG.debug("Mapped HA service delegation token for logical URI " +
-            haUri + " to namenode " + singleNNAddr);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Mapped HA service delegation token for logical URI " +
+              haUri + " to namenode " + singleNNAddr);
+        }
       }
     } else {
-      LOG.debug("No HA service delegation token found for logical URI " +
-          haUri);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("No HA service delegation token found for logical URI " +
+            haUri);
+      }
     }
   }
 

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -580,9 +580,11 @@ class BPServiceActor implements Runnable {
       long createCost = createTime - startTime;
       long sendCost = sendTime - createTime;
       dn.getMetrics().addCacheReport(sendCost);
-      LOG.debug("CacheReport of " + blockIds.size()
-          + " block(s) took " + createCost + " msec to generate and "
-          + sendCost + " msecs for RPC and NN processing");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("CacheReport of " + blockIds.size()
+            + " block(s) took " + createCost + " msec to generate and "
+            + sendCost + " msecs for RPC and NN processing");
+      }
     }
     return cmd;
   }

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetCache.java

@@ -319,9 +319,11 @@ public class FsDatasetCache {
       mappableBlockMap.put(key,
           new Value(prevValue.mappableBlock, State.UNCACHING));
       if (deferred) {
-        LOG.debug("{} is anchored, and can't be uncached now.  Scheduling it " +
-            "for uncaching in {} ",
-            key, DurationFormatUtils.formatDurationHMS(revocationPollingMs));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("{} is anchored, and can't be uncached now.  Scheduling it " +
+                  "for uncaching in {} ",
+              key, DurationFormatUtils.formatDurationHMS(revocationPollingMs));
+        }
         deferredUncachingExecutor.schedule(
             new UncachingTask(key, revocationMs),
             revocationPollingMs, TimeUnit.MILLISECONDS);

+ 14 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java

@@ -319,9 +319,11 @@ public class FileJournalManager implements JournalManager {
       Collection<EditLogInputStream> streams, long fromTxId,
       boolean inProgressOk) throws IOException {
     List<EditLogFile> elfs = matchEditLogs(sd.getCurrentDir());
-    LOG.debug(this + ": selecting input streams starting at " + fromTxId + 
-        (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +
-        "from among " + elfs.size() + " candidate file(s)");
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(this + ": selecting input streams starting at " + fromTxId +
+          (inProgressOk ? " (inProgress ok) " : " (excluding inProgress) ") +
+          "from among " + elfs.size() + " candidate file(s)");
+    }
     addStreamsToCollectionFromFiles(elfs, streams, fromTxId, inProgressOk);
   }
   
@@ -330,8 +332,10 @@ public class FileJournalManager implements JournalManager {
     for (EditLogFile elf : elfs) {
       if (elf.isInProgress()) {
         if (!inProgressOk) {
-          LOG.debug("passing over " + elf + " because it is in progress " +
-              "and we are ignoring in-progress logs.");
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("passing over " + elf + " because it is in progress " +
+                "and we are ignoring in-progress logs.");
+          }
           continue;
         }
         try {
@@ -344,9 +348,11 @@ public class FileJournalManager implements JournalManager {
       }
       if (elf.lastTxId < fromTxId) {
         assert elf.lastTxId != HdfsConstants.INVALID_TXID;
-        LOG.debug("passing over " + elf + " because it ends at " +
-            elf.lastTxId + ", but we only care about transactions " +
-            "as new as " + fromTxId);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("passing over " + elf + " because it ends at " +
+              elf.lastTxId + ", but we only care about transactions " +
+              "as new as " + fromTxId);
+        }
         continue;
       }
       EditLogFileInputStream elfis = new EditLogFileInputStream(elf.getFile(),

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -1529,7 +1529,9 @@ public class NameNode implements NameNodeStatusMXBean {
       URI defaultUri = URI.create(HdfsConstants.HDFS_URI_SCHEME + "://"
           + conf.get(DFS_NAMENODE_RPC_ADDRESS_KEY));
       conf.set(FS_DEFAULT_NAME_KEY, defaultUri.toString());
-      LOG.debug("Setting " + FS_DEFAULT_NAME_KEY + " to " + defaultUri.toString());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Setting " + FS_DEFAULT_NAME_KEY + " to " + defaultUri.toString());
+      }
     }
   }
     

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java

@@ -446,7 +446,9 @@ public class ShortCircuitCache implements Closeable {
           purgeReason = "purging replica because it is stale.";
         }
         if (purgeReason != null) {
-          LOG.debug(this + ": " + purgeReason);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(this + ": " + purgeReason);
+          }
           purge(replica);
         }
       }

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java

@@ -147,8 +147,10 @@ class FSImageLoader {
             summary.getCodec(), new BufferedInputStream(new LimitInputStream(
             fin, s.getLength())));
 
-        LOG.debug("Loading section " + s.getName() + " length: " + s.getLength
-                ());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Loading section " + s.getName() + " length: " + s.getLength
+              ());
+        }
         switch (FSImageFormatProtobuf.SectionName.fromString(s.getName())) {
           case STRING_TABLE:
             stringTable = loadStringTable(is);

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/LightWeightHashSet.java

@@ -127,8 +127,10 @@ public class LightWeightHashSet<T> implements Collection<T> {
     this.shrinkThreshold = (int) (capacity * minLoadFactor);
 
     entries = new LinkedElement[capacity];
-    LOG.debug("initial capacity=" + initialCapacity + ", max load factor= "
-        + maxLoadFactor + ", min load factor= " + minLoadFactor);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("initial capacity=" + initialCapacity + ", max load factor= "
+          + maxLoadFactor + ", min load factor= " + minLoadFactor);
+    }
   }
 
   public LightWeightHashSet() {

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java

@@ -142,7 +142,9 @@ final class TokenAspect<T extends FileSystem & Renewable> {
       if (token != null) {
         fs.setDelegationToken(token);
         addRenewAction(fs);
-        LOG.debug("Created new DT for " + token.getService());
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Created new DT for " + token.getService());
+        }
       }
       hasInitedToken = true;
     }
@@ -155,7 +157,9 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   synchronized void initDelegationToken(UserGroupInformation ugi) {
     Token<?> token = selectDelegationToken(ugi);
     if (token != null) {
-      LOG.debug("Found existing DT for " + token.getService());
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Found existing DT for " + token.getService());
+      }
       fs.setDelegationToken(token);
       hasInitedToken = true;
     }

+ 12 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -230,12 +230,16 @@ public class WebHdfsFileSystem extends FileSystem
       // refetch tokens.  even if ugi has credentials, don't attempt
       // to get another token to match hdfs/rpc behavior
       if (token != null) {
-        LOG.debug("Using UGI token: " + token);
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Using UGI token: " + token);
+        }
         canRefreshDelegationToken = false; 
       } else {
         token = getDelegationToken(null);
         if (token != null) {
-          LOG.debug("Fetched new token: " + token);
+          if(LOG.isDebugEnabled()) {
+            LOG.debug("Fetched new token: " + token);
+          }
         } else { // security is disabled
           canRefreshDelegationToken = false;
         }
@@ -250,7 +254,9 @@ public class WebHdfsFileSystem extends FileSystem
     boolean replaced = false;
     if (canRefreshDelegationToken) {
       Token<?> token = getDelegationToken(null);
-      LOG.debug("Replaced expired token: " + token);
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("Replaced expired token: " + token);
+      }
       setDelegationToken(token);
       replaced = (token != null);
     }
@@ -1196,7 +1202,9 @@ public class WebHdfsFileSystem extends FileSystem
         cancelDelegationToken(delegationToken);
       }
     } catch (IOException ioe) {
-      LOG.debug("Token cancel failed: "+ioe);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Token cancel failed: " + ioe);
+      }
     } finally {
       super.close();
     }