浏览代码

HDFS-16648. Add isDebugEnabled check for debug blockLogs in some classes (#4529)

xuzq 2 年之前
父节点
当前提交
25ccdc77af

+ 35 - 73
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -287,11 +287,8 @@ public abstract class Server {
       throw new IllegalArgumentException("ReRegistration of rpcKind: " +
           rpcKind);      
     }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("rpcKind=" + rpcKind +
-          ", rpcRequestWrapperClass=" + rpcRequestWrapperClass +
-          ", rpcInvoker=" + rpcInvoker);
-    }
+    LOG.debug("rpcKind={}, rpcRequestWrapperClass={}, rpcInvoker={}.",
+        rpcKind, rpcRequestWrapperClass, rpcInvoker);
   }
   
   public Class<? extends Writable> getRpcRequestWrapper(
@@ -1212,9 +1209,7 @@ public abstract class Server {
         deltaNanos = Time.monotonicNowNanos() - startNanos;
         details.set(Timing.RESPONSE, deltaNanos, TimeUnit.NANOSECONDS);
       } else {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Deferring response for callId: " + this.callId);
-        }
+        LOG.debug("Deferring response for callId: {}", this.callId);
       }
       return null;
     }
@@ -1711,9 +1706,7 @@ public abstract class Server {
           // If there were some calls that have not been sent out for a
           // long time, discard them.
           //
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Checking for old call responses.");
-          }
+          LOG.debug("Checking for old call responses.");
           ArrayList<RpcCall> calls;
           
           // get the list of channels from list of keys.
@@ -1813,9 +1806,8 @@ public abstract class Server {
           //
           call = responseQueue.removeFirst();
           SocketChannel channel = call.connection.channel;
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(Thread.currentThread().getName() + ": responding to " + call);
-          }
+
+          LOG.debug("{}: responding to {}.", Thread.currentThread().getName(), call);
           //
           // Send as much data as we can in the non-blocking fashion
           //
@@ -1832,10 +1824,8 @@ public abstract class Server {
             } else {
               done = false;            // more calls pending to be sent.
             }
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
-                  + " Wrote " + numBytes + " bytes.");
-            }
+            LOG.debug("{}: responding to {} Wrote {} bytes.",
+                Thread.currentThread().getName(), call, numBytes);
           } else {
             //
             // If we were unable to write the entire response out, then 
@@ -1860,10 +1850,8 @@ public abstract class Server {
                 decPending();
               }
             }
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(Thread.currentThread().getName() + ": responding to " + call
-                  + " Wrote partial " + numBytes + " bytes.");
-            }
+            LOG.debug("{}: responding to {} Wrote partial {} bytes.",
+                Thread.currentThread().getName(), call, numBytes);
           }
           error = false;              // everything went off well
         }
@@ -2209,13 +2197,11 @@ public abstract class Server {
         
         if (saslServer != null && saslServer.isComplete()) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("SASL server context established. Negotiated QoP is "
-                + saslServer.getNegotiatedProperty(Sasl.QOP));
+            LOG.debug("SASL server context established. Negotiated QoP is {}.",
+                saslServer.getNegotiatedProperty(Sasl.QOP));
           }
           user = getAuthorizedUgi(saslServer.getAuthorizationID());
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("SASL server successfully authenticated client: " + user);
-          }
+          LOG.debug("SASL server successfully authenticated client: {}.", user);
           rpcMetrics.incrAuthenticationSuccesses();
           AUDITLOG.info(AUTH_SUCCESSFUL_FOR + user + " from " + toString());
           saslContextEstablished = true;
@@ -2320,10 +2306,8 @@ public abstract class Server {
         throw new SaslException("Client did not send a token");
       }
       byte[] saslToken = saslMessage.getToken().toByteArray();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Have read input token of size " + saslToken.length
-            + " for processing by saslServer.evaluateResponse()");
-      }
+      LOG.debug("Have read input token of size {} for processing by saslServer.evaluateResponse()",
+          saslToken.length);
       saslToken = saslServer.evaluateResponse(saslToken);
       return buildSaslResponse(
           saslServer.isComplete() ? SaslState.SUCCESS : SaslState.CHALLENGE,
@@ -2338,9 +2322,8 @@ public abstract class Server {
 
     private RpcSaslProto buildSaslResponse(SaslState state, byte[] replyToken) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Will send " + state + " token of size "
-            + ((replyToken != null) ? replyToken.length : null)
-            + " from saslServer.");
+        LOG.debug("Will send {} token of size {} from saslServer.", state,
+            ((replyToken != null) ? replyToken.length : null));
       }
       RpcSaslProto.Builder response = RpcSaslProto.newBuilder();
       response.setState(state);
@@ -2664,10 +2647,8 @@ public abstract class Server {
      */    
     private void unwrapPacketAndProcessRpcs(byte[] inBuf)
         throws IOException, InterruptedException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Have read input token of size " + inBuf.length
-            + " for processing by saslServer.unwrap()");
-      }
+      LOG.debug("Have read input token of size {} for processing by saslServer.unwrap()",
+          inBuf.length);
       inBuf = saslServer.unwrap(inBuf, 0, inBuf.length);
       ReadableByteChannel ch = Channels.newChannel(new ByteArrayInputStream(
           inBuf));
@@ -2729,9 +2710,7 @@ public abstract class Server {
             getMessage(RpcRequestHeaderProto.getDefaultInstance(), buffer);
         callId = header.getCallId();
         retry = header.getRetryCount();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(" got #" + callId);
-        }
+        LOG.debug(" got #{}", callId);
         checkRpcHeaders(header);
 
         if (callId < 0) { // callIds typically used during connection setup
@@ -2746,11 +2725,8 @@ public abstract class Server {
       } catch (RpcServerException rse) {
         // inform client of error, but do not rethrow else non-fatal
         // exceptions will close connection!
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(Thread.currentThread().getName() +
-              ": processOneRpc from client " + this +
-              " threw exception [" + rse + "]");
-        }
+        LOG.debug("{}: processOneRpc from client {} threw exception [{}]",
+            Thread.currentThread().getName(), this, rse);
         // use the wrapped exception if there is one.
         Throwable t = (rse.getCause() != null) ? rse.getCause() : rse;
         final RpcCall call = new RpcCall(this, callId, retry);
@@ -2962,9 +2938,7 @@ public abstract class Server {
           ProxyUsers.authorize(user, this.getHostAddress());
         }
         authorize(user, protocolName, getHostInetAddress());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Successfully authorized " + connectionContext);
-        }
+        LOG.debug("Successfully authorized {}.", connectionContext);
         rpcMetrics.incrAuthorizationSuccesses();
       } catch (AuthorizationException ae) {
         LOG.info("Connection from " + this
@@ -3081,7 +3055,7 @@ public abstract class Server {
 
     @Override
     public void run() {
-      LOG.debug(Thread.currentThread().getName() + ": starting");
+      LOG.debug("{}: starting", Thread.currentThread().getName());
       SERVER.set(Server.this);
       while (running) {
         TraceScope traceScope = null;
@@ -3115,9 +3089,7 @@ public abstract class Server {
             call = null;
             continue;
           }
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
-          }
+          LOG.debug("{}: {} for RpcKind {}.", Thread.currentThread().getName(), call, call.rpcKind);
           CurCall.set(call);
           if (call.span != null) {
             traceScope = tracer.activateSpan(call.span);
@@ -3152,15 +3124,14 @@ public abstract class Server {
           IOUtils.cleanupWithLogger(LOG, traceScope);
           if (call != null) {
             updateMetrics(call, startTimeNanos, connDropped);
-            ProcessingDetails.LOG.debug(
-                "Served: [{}]{} name={} user={} details={}",
+            ProcessingDetails.LOG.debug("Served: [{}]{} name={} user={} details={}",
                 call, (call.isResponseDeferred() ? ", deferred" : ""),
                 call.getDetailedMetricsName(), call.getRemoteUser(),
                 call.getProcessingDetails());
           }
         }
       }
-      LOG.debug(Thread.currentThread().getName() + ": exiting");
+      LOG.debug("{}: exiting", Thread.currentThread().getName());
     }
 
     private void requeueCall(Call call)
@@ -3389,14 +3360,13 @@ public abstract class Server {
             " authentication requires a secret manager");
       } 
     } else if (secretManager != null) {
-      LOG.debug(AuthenticationMethod.TOKEN +
-          " authentication enabled for secret manager");
+      LOG.debug("{} authentication enabled for secret manager", AuthenticationMethod.TOKEN);
       // most preferred, go to the front of the line!
       authMethods.add(AuthenticationMethod.TOKEN.getAuthMethod());
     }
     authMethods.add(confAuthenticationMethod.getAuthMethod());        
     
-    LOG.debug("Server accepts auth methods:" + authMethods);
+    LOG.debug("Server accepts auth methods:{}", authMethods);
     return authMethods;
   }
   
@@ -3556,9 +3526,7 @@ public abstract class Server {
       synchronized (call.connection.saslServer) {
         token = call.connection.saslServer.wrap(token, 0, token.length);
       }
-      if (LOG.isDebugEnabled())
-        LOG.debug("Adding saslServer wrapped token of size " + token.length
-            + " as call response.");
+      LOG.debug("Adding saslServer wrapped token of size {} as call response.", token.length);
       // rebuild with sasl header and payload
       RpcResponseHeaderProto saslHeader = RpcResponseHeaderProto.newBuilder()
           .setCallId(AuthProtocol.SASL.callId)
@@ -4004,11 +3972,8 @@ public abstract class Server {
       Connection connection = new Connection(channel, Time.now(),
           ingressPort, isOnAuxiliaryPort);
       add(connection);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Server connection from " + connection +
-            "; # active connections: " + size() +
-            "; # queued calls: " + callQueue.size());
-      }      
+      LOG.debug("Server connection from {}; # active connections: {}; # queued calls: {}.",
+          connection, size(), callQueue.size());
       return connection;
     }
     
@@ -4016,9 +3981,8 @@ public abstract class Server {
       boolean exists = remove(connection);
       if (exists) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug(Thread.currentThread().getName() +
-              ": disconnecting client " + connection +
-              ". Number of active connections: "+ size());
+          LOG.debug("{}: disconnecting client {}. Number of active connections: {}.",
+              Thread.currentThread().getName(), connection, size());
         }
         // only close if actually removed to avoid double-closing due
         // to possible races
@@ -4080,9 +4044,7 @@ public abstract class Server {
           if (!running) {
             return;
           }
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(Thread.currentThread().getName()+": task running");
-          }
+          LOG.debug("{}: task running", Thread.currentThread().getName());
           try {
             closeIdle(false);
           } finally {

+ 7 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java

@@ -522,8 +522,7 @@ public class NetworkTopology {
       }
     }
     if (numOfDatanodes <= 0) {
-      LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\")."
-              + " numOfDatanodes={}",
+      LOG.debug("Failed to find datanode (scope=\"{}\" excludedScope=\"{}\"). numOfDatanodes={}",
           scope, excludedScope, numOfDatanodes);
       return null;
     }
@@ -539,10 +538,12 @@ public class NetworkTopology {
         netlock.readLock().unlock();
       }
     }
-    LOG.debug("Choosing random from {} available nodes on node {},"
-        + " scope={}, excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
-        availableNodes, innerNode, scope, excludedScope, excludedNodes,
-        numOfDatanodes);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Choosing random from {} available nodes on node {}, scope={},"
+              + " excludedScope={}, excludeNodes={}. numOfDatanodes={}.",
+          availableNodes, innerNode, scope, excludedScope, excludedNodes,
+          numOfDatanodes);
+    }
     Node ret = null;
     if (availableNodes > 0) {
       ret = chooseRandom(innerNode, node, excludedNodes, numOfDatanodes,

+ 5 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -479,10 +479,9 @@ public class QuorumJournalManager implements JournalManager {
     LOG.info("Successfully started new epoch " + loggers.getEpoch());
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("newEpoch(" + loggers.getEpoch() + ") responses:\n" +
-        QuorumCall.mapToString(resps));
+      LOG.debug("newEpoch({}) responses:\n{}", loggers.getEpoch(), QuorumCall.mapToString(resps));
     }
-    
+
     long mostRecentSegmentTxId = Long.MIN_VALUE;
     for (NewEpochResponseProto r : resps.values()) {
       if (r.hasLastSegmentTxId()) {
@@ -518,10 +517,7 @@ public class QuorumJournalManager implements JournalManager {
     // the cache used for RPC calls is not enabled; fall back to using the
     // streaming mechanism to serve such requests
     if (inProgressOk && inProgressTailingEnabled) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Tailing edits starting from txn ID " + fromTxnId +
-            " via RPC mechanism");
-      }
+      LOG.debug("Tailing edits starting from txn ID {} via RPC mechanism", fromTxnId);
       try {
         Collection<EditLogInputStream> rpcStreams = new ArrayList<>();
         selectRpcInputStreams(rpcStreams, fromTxnId, onlyDurableTxns);
@@ -585,8 +581,8 @@ public class QuorumJournalManager implements JournalManager {
     int maxAllowedTxns = !onlyDurableTxns ? highestTxnCount :
         responseCounts.get(responseCounts.size() - loggers.getMajoritySize());
     if (maxAllowedTxns == 0) {
-      LOG.debug("No new edits available in logs; requested starting from " +
-          "ID {}", fromTxnId);
+      LOG.debug("No new edits available in logs; requested starting from ID {}",
+          fromTxnId);
       return;
     }
     LogAction logAction = selectInputStreamLogHelper.record(fromTxnId);

+ 69 - 76
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -1541,6 +1541,7 @@ public class BlockManager implements BlockStatsMXBean {
       if (LOG.isDebugEnabled()) {
         LOG.debug("blocks = {}", java.util.Arrays.asList(blocks));
       }
+
       final AccessMode mode = needBlockToken? BlockTokenIdentifier.AccessMode.READ: null;
 
       LocatedBlockBuilder locatedBlocks = providedStorageMap
@@ -1873,8 +1874,7 @@ public class BlockManager implements BlockStatsMXBean {
     }
 
     if (storage == null) {
-      blockLog.debug("BLOCK* findAndMarkBlockAsCorrupt: {} not found on {}",
-          blk, dn);
+      blockLog.debug("BLOCK* findAndMarkBlockAsCorrupt: {} not found on {}", blk, dn);
       return;
     }
     markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock,
@@ -1893,7 +1893,7 @@ public class BlockManager implements BlockStatsMXBean {
       DatanodeStorageInfo storageInfo,
       DatanodeDescriptor node) throws IOException {
     if (b.getStored().isDeleted()) {
-      if(blockLog.isDebugEnabled()) {
+      if (blockLog.isDebugEnabled()) {
         blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
             " corrupt as it does not belong to any file", b);
       }
@@ -1977,7 +1977,7 @@ public class BlockManager implements BlockStatsMXBean {
 
     // Check how many copies we have of the block
     if (nr.replicasOnStaleNodes() > 0 && !deleteCorruptReplicaImmediately) {
-      if(blockLog.isDebugEnabled()) {
+      if (blockLog.isDebugEnabled()) {
         blockLog.debug("BLOCK* invalidateBlocks: postponing " +
             "invalidation of {} on {} because {} replica(s) are located on " +
             "nodes with potentially out-of-date block reports", b, dn,
@@ -1990,8 +1990,7 @@ public class BlockManager implements BlockStatsMXBean {
       // function and know there are enough live replicas, so we can delete it.
       addToInvalidates(b.getCorrupted(), dn);
       removeStoredBlock(b.getStored(), node);
-      blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.",
-          b, dn);
+      blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.", b, dn);
       return true;
     }
   }
@@ -2156,13 +2155,11 @@ public class BlockManager implements BlockStatsMXBean {
           for (DatanodeStorageInfo target : targets) {
             targetList.append(' ').append(target.getDatanodeDescriptor());
           }
-          blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNodes(),
-              rw.getBlock(), targetList);
+          blockLog.debug("BLOCK* ask {} to replicate {} to {}",
+              rw.getSrcNodes(), rw.getBlock(), targetList);
         }
       }
-
-      blockLog.debug(
-          "BLOCK* neededReconstruction = {} pendingReconstruction = {}",
+      blockLog.debug("BLOCK* neededReconstruction = {} pendingReconstruction = {}",
           neededReconstruction.size(), pendingReconstruction.size());
     }
 
@@ -2201,7 +2198,7 @@ public class BlockManager implements BlockStatsMXBean {
         liveBlockIndices, liveBusyBlockIndices, excludeReconstructed, priority);
     short requiredRedundancy = getExpectedLiveRedundancyNum(block,
         numReplicas);
-    if(srcNodes == null || srcNodes.length == 0) {
+    if (srcNodes == null || srcNodes.length == 0) {
       // block can not be reconstructed from any node
       LOG.debug("Block {} cannot be reconstructed from any node", block);
       NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
@@ -2225,10 +2222,8 @@ public class BlockManager implements BlockStatsMXBean {
     int pendingNum = pendingReconstruction.getNumReplicas(block);
     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
       neededReconstruction.remove(block, priority);
-      if(blockLog.isDebugEnabled()) {
-        blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
-            " it has enough replicas", block);
-      }
+      blockLog.debug("BLOCK* Removing {} from neededReconstruction as it has enough replicas",
+          block);
       NameNode.getNameNodeMetrics().incNumTimesReReplicationNotScheduled();
       return null;
     }
@@ -2328,10 +2323,8 @@ public class BlockManager implements BlockStatsMXBean {
     if (hasEnoughEffectiveReplicas(block, numReplicas, pendingNum)) {
       neededReconstruction.remove(block, priority);
       rw.resetTargets();
-      if(blockLog.isDebugEnabled()) {
-        blockLog.debug("BLOCK* Removing {} from neededReconstruction as" +
-            " it has enough replicas", block);
-      }
+      blockLog.debug("BLOCK* Removing {} from neededReconstruction as it has enough replicas",
+          block);
       return false;
     }
 
@@ -2362,10 +2355,8 @@ public class BlockManager implements BlockStatsMXBean {
     // The reason we use 'pending' is so we can retry
     // reconstructions that fail after an appropriate amount of time.
     pendingReconstruction.increment(block, targets);
-    if(blockLog.isDebugEnabled()) {
-      blockLog.debug("BLOCK* block {} is moved from neededReconstruction to "
-          + "pendingReconstruction", block);
-    }
+    blockLog.debug("BLOCK* block {} is moved from neededReconstruction to pendingReconstruction",
+        block);
 
     int numEffectiveReplicas = numReplicas.liveReplicas() + pendingNum;
     // remove from neededReconstruction
@@ -2758,9 +2749,11 @@ public class BlockManager implements BlockStatsMXBean {
       removeBlock(b);
     }
     if (trackBlockCounts) {
-      LOG.debug("Adjusting safe-mode totals for deletion."
-          + "decreasing safeBlocks by {}, totalBlocks by {}",
-          numRemovedSafe, numRemovedComplete);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Adjusting safe-mode totals for deletion."
+                + "decreasing safeBlocks by {}, totalBlocks by {}",
+            numRemovedSafe, numRemovedComplete);
+      }
       bmSafeMode.adjustBlockTotals(-numRemovedSafe, -numRemovedComplete);
     }
   }
@@ -2913,13 +2906,11 @@ public class BlockManager implements BlockStatsMXBean {
       namesystem.writeUnlock("processReport");
     }
 
-    if(blockLog.isDebugEnabled()) {
+    if (blockLog.isDebugEnabled()) {
       for (Block b : invalidatedBlocks) {
-        if(blockLog.isDebugEnabled()) {
-          blockLog.debug("BLOCK* processReport 0x{} with lease ID 0x{}: {} on node {} size {} " +
-                          "does not belong to any file.", strBlockReportId, fullBrLeaseId, b,
-                           node, b.getNumBytes());
-        }
+        blockLog.debug("BLOCK* processReport 0x{} with lease ID 0x{}: {} on node {} size {} " +
+                "does not belong to any file.", strBlockReportId, fullBrLeaseId, b,
+            node, b.getNumBytes());
       }
     }
 
@@ -2951,9 +2942,10 @@ public class BlockManager implements BlockStatsMXBean {
           node.setLastBlockReportTime(now());
           node.setLastBlockReportMonotonic(Time.monotonicNow());
         }
-        LOG.debug("Processing RPC with index {} out of total {} RPCs in "
-                + "processReport 0x{}", context.getCurRpc(),
-            context.getTotalRpcs(), Long.toHexString(context.getReportId()));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Processing RPC with index {} out of total {} RPCs in processReport 0x{}",
+              context.getCurRpc(), context.getTotalRpcs(), Long.toHexString(context.getReportId()));
+        }
       }
     } finally {
       namesystem.writeUnlock("removeBRLeaseIfNeeded");
@@ -2978,14 +2970,16 @@ public class BlockManager implements BlockStatsMXBean {
 
         BlockInfo bi = getStoredBlock(b);
         if (bi == null) {
-          LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
-              "Postponed mis-replicated block {} no longer found " +
-              "in block map.", b);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
+                "Postponed mis-replicated block {} no longer found " +
+                "in block map.", b);
+          }
           continue;
         }
         MisReplicationResult res = processMisReplicatedBlock(bi);
-        LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
-            "Re-scanned block {}, result is {}", b, res);
+        LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: Re-scanned block {}, result is {}",
+            b, res);
         if (res == MisReplicationResult.POSTPONE) {
           rescannedMisreplicatedBlocks.add(b);
         }
@@ -3077,7 +3071,7 @@ public class BlockManager implements BlockStatsMXBean {
         }
       }
       if (isCorrupt) {
-        if(blockLog.isDebugEnabled()) {
+        if (blockLog.isDebugEnabled()) {
           blockLog.debug("BLOCK* markBlockReplicasAsCorrupt: mark block replica" +
               " {} on {} as corrupt because the dn is not in the new committed " +
               "storage list.", b, storage.getDatanodeDescriptor());
@@ -3113,6 +3107,7 @@ public class BlockManager implements BlockStatsMXBean {
             iblk.getBlockName(), storageInfo.getDatanodeDescriptor(),
             iblk.getNumBytes(), reportedState);
       }
+
       if (shouldPostponeBlocksFromFuture && isGenStampInFuture(iblk)) {
         queueReportedBlock(storageInfo, iblk, reportedState,
             QUEUE_REASON_FUTURE_GENSTAMP);
@@ -3329,9 +3324,11 @@ public class BlockManager implements BlockStatsMXBean {
       ReplicaState reportedState, String reason) {
     assert shouldPostponeBlocksFromFuture;
 
-    LOG.debug("Queueing reported block {} in state {}" +
-            " from datanode {} for later processing because {}.",
-        block, reportedState, storageInfo.getDatanodeDescriptor(), reason);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Queueing reported block {} in state {}" +
+              " from datanode {} for later processing because {}.",
+          block, reportedState, storageInfo.getDatanodeDescriptor(), reason);
+    }
     pendingDNMessages.enqueueReportedBlock(storageInfo, block, reportedState);
   }
 
@@ -3598,11 +3595,8 @@ public class BlockManager implements BlockStatsMXBean {
     }
     if (storedBlock == null || storedBlock.isDeleted()) {
       // If this block does not belong to anyfile, then we are done.
-      if(blockLog.isDebugEnabled()) {
-        blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not" +
-            " belong to any file", block, node, block.getNumBytes());
-      }
-
+      blockLog.debug("BLOCK* addStoredBlock: {} on {} size {} but it does not belong to any file",
+          block, node, block.getNumBytes());
       // we could add this block to invalidate set of this datanode.
       // it will happen in next block report otherwise.
       return block;
@@ -3630,7 +3624,7 @@ public class BlockManager implements BlockStatsMXBean {
       corruptReplicas.removeFromCorruptReplicasMap(block, node,
           Reason.GENSTAMP_MISMATCH);
       curReplicaDelta = 0;
-      if(blockLog.isDebugEnabled()) {
+      if (blockLog.isDebugEnabled()) {
         blockLog.debug("BLOCK* addStoredBlock: Redundant addStoredBlock request"
                 + " received for {} on node {} size {}", storedBlock, node,
             storedBlock.getNumBytes());
@@ -3735,10 +3729,8 @@ public class BlockManager implements BlockStatsMXBean {
           removedFromBlocksMap = false;
         }
       } catch (IOException e) {
-        if(blockLog.isDebugEnabled()) {
-          blockLog.debug("invalidateCorruptReplicas error in deleting bad block"
-              + " {} on {}", blk, node, e);
-        }
+        blockLog.debug("invalidateCorruptReplicas error in deleting bad block {} on {}",
+            blk, node, e);
         removedFromBlocksMap = false;
       }
     }
@@ -3920,8 +3912,8 @@ public class BlockManager implements BlockStatsMXBean {
             BlockInfo blk = iter.next();
             MisReplicationResult r = processMisReplicatedBlock(blk);
             processed++;
-            LOG.debug("BLOCK* processMisReplicatedBlocks: " +
-                    "Re-scanned block {}, result is {}", blk, r);
+            LOG.debug("BLOCK* processMisReplicatedBlocks: Re-scanned block {}, result is {}",
+                blk, r);
           }
         } finally {
           namesystem.writeUnlock("processMisReplicatedBlocks");
@@ -4187,10 +4179,8 @@ public class BlockManager implements BlockStatsMXBean {
     //
     final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen);
     addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor());
-    if(blockLog.isDebugEnabled()) {
-      blockLog.debug("BLOCK* chooseExcessRedundancies: "
-          + "({}, {}) is added to invalidated blocks set", chosen, storedBlock);
-    }
+    blockLog.debug("BLOCK* chooseExcessRedundancies: ({}, {}) is added to invalidated blocks set",
+        chosen, storedBlock);
   }
 
   private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block,
@@ -4212,8 +4202,8 @@ public class BlockManager implements BlockStatsMXBean {
     assert (namesystem.hasWriteLock());
     {
       if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
-        blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
-            " removed from node {}", storedBlock, node);
+        blockLog.debug("BLOCK* removeStoredBlock: {} has already been removed from node {}",
+            storedBlock, node);
         return;
       }
 
@@ -4225,8 +4215,10 @@ public class BlockManager implements BlockStatsMXBean {
         removed |= node.getCached().remove(cblock);
         removed |= node.getPendingUncached().remove(cblock);
         if (removed) {
-          blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
-              + "related lists on node {}", storedBlock, node);
+          if (blockLog.isDebugEnabled()) {
+            blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
+                + "related lists on node {}", storedBlock, node);
+          }
         }
       }
 
@@ -4251,8 +4243,9 @@ public class BlockManager implements BlockStatsMXBean {
     for (ReplicaUnderConstruction r : staleReplicas) {
       removeStoredBlock(block,
           r.getExpectedStorageLocation().getDatanodeDescriptor());
-      blockLog.debug("BLOCK* Removing stale replica {} of {}", r,
-              Block.toString(r));
+      if (blockLog.isDebugEnabled()) {
+        blockLog.debug("BLOCK* Removing stale replica {} of {}", r, Block.toString(r));
+      }
     }
   }
   /**
@@ -4380,10 +4373,8 @@ public class BlockManager implements BlockStatsMXBean {
           maxNumBlocksToLog, numBlocksLogged);
     }
     for (Block b : toInvalidate) {
-      if(blockLog.isDebugEnabled()) {
-        blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not " +
-            "belong to any file", b, node, b.getNumBytes());
-      }
+      blockLog.debug("BLOCK* addBlock: block {} on node {} size {} does not belong to any file",
+          b, node, b.getNumBytes());
       addToInvalidates(b, node);
     }
     for (BlockToMarkCorrupt b : toCorrupt) {
@@ -4464,7 +4455,7 @@ public class BlockManager implements BlockStatsMXBean {
       blockLog.debug("BLOCK* block {}: {} is received from {}",
           rdbi.getStatus(), rdbi.getBlock(), node);
     }
-    if(blockLog.isDebugEnabled()) {
+    if (blockLog.isDebugEnabled()) {
       blockLog.debug("*BLOCK* NameNode.processIncrementalBlockReport: from "
               + "{} receiving: {}, received: {}, deleted: {}", node, receiving,
           received, deleted);
@@ -4843,8 +4834,10 @@ public class BlockManager implements BlockStatsMXBean {
     } finally {
       namesystem.writeUnlock("invalidateWorkForOneNode");
     }
-    blockLog.debug("BLOCK* {}: ask {} to delete {}", getClass().getSimpleName(),
-        dn, toInvalidate);
+    if (blockLog.isDebugEnabled()) {
+      blockLog.debug("BLOCK* {}: ask {} to delete {}",
+          getClass().getSimpleName(), dn, toInvalidate);
+    }
     return toInvalidate.size();
   }
 
@@ -5110,8 +5103,8 @@ public class BlockManager implements BlockStatsMXBean {
           }
         }
         if (isSleep) {
-          LOG.debug("Clear markedDeleteQueue over {}" +
-              " millisecond to release the write lock", deleteBlockLockTimeMs);
+          LOG.debug("Clear markedDeleteQueue over {} millisecond to release the write lock",
+              deleteBlockLockTimeMs);
         }
         try {
           Thread.sleep(deleteBlockUnlockIntervalTimeMs);

+ 16 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -238,8 +238,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       return getPipeline(writer,
           results.toArray(new DatanodeStorageInfo[results.size()]));
     } catch (NotEnoughReplicasException nr) {
-      LOG.debug("Failed to choose with favored nodes (={}), disregard favored"
-          + " nodes hint and retry.", favoredNodes, nr);
+      LOG.debug("Failed to choose with favored nodes (={}), disregard favored nodes hint and retry",
+          favoredNodes, nr);
       // Fall back to regular block placement disregarding favored nodes hint
       return chooseTarget(src, numOfReplicas, writer, 
           new ArrayList<DatanodeStorageInfo>(numOfReplicas), false, 
@@ -715,17 +715,19 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
         DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor();
         if (nextNode != localMachine) {
           if (LOG.isDebugEnabled()) {
-            LOG.debug("Failed to choose from local rack (location = " + localRack
-                + "), retry with the rack of the next replica (location = "
-                + nextNode.getNetworkLocation() + ")", e);
+            LOG.debug("Failed to choose from local rack (location = {}), retry with the rack "
+                + "of the next replica (location = {})", localRack,
+                nextNode.getNetworkLocation(), e);
           }
           return chooseFromNextRack(nextNode, excludedNodes, blocksize,
               maxNodesPerRack, results, avoidStaleNodes, storageTypes);
         }
       }
 
-      LOG.debug("Failed to choose from local rack (location = {}); the second"
-          + " replica is not found, retry choosing randomly", localRack, e);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Failed to choose from local rack (location = {}); the second"
+            + " replica is not found, retry choosing randomly", localRack, e);
+      }
 
       //the second replica is not found, randomly choose one from the network
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
@@ -745,9 +747,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       return chooseRandom(nextRack, excludedNodes, blocksize, maxNodesPerRack,
           results, avoidStaleNodes, storageTypes);
     } catch (NotEnoughReplicasException e) {
-      LOG.debug("Failed to choose from the next rack (location = {}), "
-          + "retry choosing randomly", nextRack, e);
-        // otherwise randomly choose one from the network
+      LOG.debug("Failed to choose from the next rack (location = {}), retry choosing randomly",
+          nextRack, e);
+
+      // otherwise randomly choose one from the network
       return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
           maxNodesPerRack, results, avoidStaleNodes, storageTypes);
     }
@@ -775,10 +778,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
           excludedNodes, blocksize, maxReplicasPerRack, results,
           avoidStaleNodes, storageTypes);
     } catch (NotEnoughReplicasException e) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Failed to choose remote rack (location = ~"
-            + localMachine.getNetworkLocation() + "), fallback to local rack", e);
-      }
+      LOG.debug("Failed to choose remote rack (location = ~{}), fallback to local rack",
+          localMachine.getNetworkLocation(), e);
       chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
                    localMachine.getNetworkLocation(), excludedNodes, blocksize, 
                    maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
@@ -1276,8 +1277,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       firstOne = false;
       if (cur == null) {
         LOG.debug(
-            "No excess replica can be found. excessTypes: {}. "
-                + "moreThanOne: {}. exactlyOne: {}.",
+            "No excess replica can be found. excessTypes: {}. moreThanOne: {}. exactlyOne: {}.",
             excessTypes, moreThanOne, exactlyOne);
         break;
       }

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java

@@ -152,10 +152,12 @@ public class DatanodeAdminManager {
     executor.scheduleWithFixedDelay(monitor, intervalSecs, intervalSecs,
         TimeUnit.SECONDS);
 
-    LOG.debug("Activating DatanodeAdminManager with interval {} seconds, " +
-            "{} max blocks per interval, " +
-            "{} max concurrently tracked nodes.", intervalSecs,
-        blocksPerInterval, maxConcurrentTrackedNodes);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Activating DatanodeAdminManager with interval {} seconds, " +
+              "{} max blocks per interval, " +
+              "{} max concurrently tracked nodes.", intervalSecs,
+          blocksPerInterval, maxConcurrentTrackedNodes);
+    }
   }
 
   /**

+ 19 - 34
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -842,9 +842,7 @@ public class DatanodeManager {
     decrementVersionCount(nodeInfo.getSoftwareVersion());
     blockManager.getBlockReportLeaseManager().unregister(nodeInfo);
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("remove datanode " + nodeInfo);
-    }
+    LOG.debug("remove datanode {}.", nodeInfo);
     blockManager.checkSafeMode();
   }
 
@@ -906,8 +904,8 @@ public class DatanodeManager {
     resolveUpgradeDomain(node);
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug(getClass().getSimpleName() + ".addDatanode: "
-          + "node " + node + " is added to datanodeMap.");
+      LOG.debug("{}.addDatanode: node {} is added to datanodeMap.",
+          getClass().getSimpleName(), node);
     }
   }
 
@@ -918,9 +916,8 @@ public class DatanodeManager {
       host2DatanodeMap.remove(datanodeMap.remove(key));
     }
     if (LOG.isDebugEnabled()) {
-      LOG.debug(getClass().getSimpleName() + ".wipeDatanode("
-          + node + "): storage " + key 
-          + " is removed from datanodeMap.");
+      LOG.debug("{}.wipeDatanode({}): storage {} is removed from datanodeMap.",
+          getClass().getSimpleName(), node, key);
     }
   }
 
@@ -1189,10 +1186,7 @@ public class DatanodeManager {
           // The same datanode has been just restarted to serve the same data 
           // storage. We do not need to remove old data blocks, the delta will
           // be calculated on the next block report from the datanode
-          if(NameNode.stateChangeLog.isDebugEnabled()) {
-            NameNode.stateChangeLog.debug("BLOCK* registerDatanode: "
-                + "node restarted.");
-          }
+          NameNode.stateChangeLog.debug("BLOCK* registerDatanode: node restarted.");
         } else {
           // nodeS is found
           /* The registering datanode is a replacement node for the existing 
@@ -1535,9 +1529,11 @@ public class DatanodeManager {
             "now be replicated cross-rack";
         LOG.info(message);
       } else {
-        message += "Not checking for mis-replicated blocks because this NN is " +
-            "not yet processing repl queues.";
-        LOG.debug(message);
+        if (LOG.isDebugEnabled()) {
+          message += "Not checking for mis-replicated blocks because this NN "
+              + "is not yet processing repl queues.";
+          LOG.debug(message);
+        }
       }
       hasClusterEverBeenMultiRack = true;
       if (blockManager.isPopulatingReplQueues()) {
@@ -1659,11 +1655,9 @@ public class DatanodeManager {
     }
 
     if (LOG.isDebugEnabled()) {
-      LOG.debug("getDatanodeListForReport with " +
-          "includedNodes = " + hostConfigManager.getIncludes() +
-          ", excludedNodes = " + hostConfigManager.getExcludes() +
-          ", foundNodes = " + foundNodes +
-          ", nodes = " + nodes);
+      LOG.debug("getDatanodeListForReport with includedNodes = {}, excludedNodes = {}"
+              + ", foundNodes = {}, nodes = {}.", hostConfigManager.getIncludes(),
+          hostConfigManager.getExcludes(), foundNodes, nodes);
     }
     return nodes;
   }
@@ -1847,10 +1841,8 @@ public class DatanodeManager {
           (double) (totalReplicateBlocks * maxTransfers) / totalBlocks);
       int numECTasks = (int) Math.ceil(
           (double) (totalECBlocks * maxTransfers) / totalBlocks);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Pending replication tasks: " + numReplicationTasks
-            + " erasure-coded tasks: " + numECTasks);
-      }
+      LOG.debug("Pending replication tasks: {} erasure-coded tasks: {}.",
+          numReplicationTasks, numECTasks);
       // check pending replication tasks
       List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
           numReplicationTasks);
@@ -1906,9 +1898,7 @@ public class DatanodeManager {
     if (slowPeerTracker.isSlowPeerTrackerEnabled()) {
       final Map<String, OutlierMetrics> slowPeersMap = slowPeers.getSlowPeers();
       if (!slowPeersMap.isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("DataNode " + nodeReg + " reported slow peers: " + slowPeersMap);
-        }
+        LOG.debug("DataNode {} reported slow peers: {}.", nodeReg, slowPeersMap);
         for (Map.Entry<String, OutlierMetrics> slowNodeEntry : slowPeersMap.entrySet()) {
           slowPeerTracker.addReport(slowNodeEntry.getKey(), nodeReg.getIpcAddr(false),
               slowNodeEntry.getValue());
@@ -1918,10 +1908,7 @@ public class DatanodeManager {
 
     if (slowDiskTracker != null) {
       if (!slowDisks.getSlowDisks().isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("DataNode " + nodeReg + " reported slow disks: " +
-              slowDisks.getSlowDisks());
-        }
+        LOG.debug("DataNode {} reported slow disks: {}.", nodeReg, slowDisks.getSlowDisks());
         slowDiskTracker.addSlowDiskReport(nodeReg.getIpcAddr(false), slowDisks);
       }
       slowDiskTracker.checkAndUpdateReportIfNecessary();
@@ -1950,9 +1937,7 @@ public class DatanodeManager {
       StorageReport[] reports, long cacheCapacity,
       long cacheUsed, int xceiverCount, int failedVolumes,
       VolumeFailureSummary volumeFailureSummary) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Received handleLifeline from nodeReg = " + nodeReg);
-    }
+    LOG.debug("Received handleLifeline from nodeReg = {}.", nodeReg);
     DatanodeDescriptor nodeinfo = getDatanode(nodeReg);
     if (nodeinfo == null || !nodeinfo.isRegistered()) {
       // This can happen if the lifeline message comes when DataNode is either

+ 4 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java

@@ -976,9 +976,8 @@ public class CacheManager {
   public final void processCacheReport(final DatanodeID datanodeID,
       final List<Long> blockIds) throws IOException {
     if (!enabled) {
-      LOG.debug("Ignoring cache report from {} because {} = false. " +
-              "number of blocks: {}", datanodeID,
-              DFS_NAMENODE_CACHING_ENABLED_KEY, blockIds.size());
+      LOG.debug("Ignoring cache report from {} because {} = false. number of blocks: {}",
+          datanodeID, DFS_NAMENODE_CACHING_ENABLED_KEY, blockIds.size());
       return;
     }
     namesystem.writeLock();
@@ -1003,9 +1002,8 @@ public class CacheManager {
     if (metrics != null) {
       metrics.addCacheBlockReport((int) (endTime - startTime));
     }
-    LOG.debug("Processed cache report from {}, blocks: {}, " +
-        "processing time: {} msecs", datanodeID, blockIds.size(), 
-        (endTime - startTime));
+    LOG.debug("Processed cache report from {}, blocks: {}, processing time: {} msecs",
+        datanodeID, blockIds.size(), (endTime - startTime));
   }
 
   private void processCacheReportImpl(final DatanodeDescriptor datanode,

+ 9 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -649,7 +649,7 @@ public class FSImage implements Closeable {
    */
   void reloadFromImageFile(File file, FSNamesystem target) throws IOException {
     target.clear();
-    LOG.debug("Reloading namespace from " + file);
+    LOG.debug("Reloading namespace from {}.", file);
     loadFSImage(file, target, null, false);
   }
 
@@ -728,7 +728,7 @@ public class FSImage implements Closeable {
     }
  
     for (EditLogInputStream l : editStreams) {
-      LOG.debug("Planning to load edit log stream: " + l);
+      LOG.debug("Planning to load edit log stream: {}.", l);
     }
     if (!editStreams.iterator().hasNext()) {
       LOG.info("No edit log streams selected.");
@@ -892,8 +892,10 @@ public class FSImage implements Closeable {
       FSNamesystem target, long maxTxnsToRead,
       StartupOption startOpt, MetaRecoveryContext recovery)
       throws IOException {
-    LOG.debug("About to load edits:\n  " + Joiner.on("\n  ").join(editStreams));
-    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("About to load edits:\n  {}.", Joiner.on("\n  ").join(editStreams));
+    }
+
     long prevLastAppliedTxId = lastAppliedTxId;
     long remainingReadTxns = maxTxnsToRead;
     try {    
@@ -1348,10 +1350,10 @@ public class FSImage implements Closeable {
     final File fromFile = NNStorage.getStorageFile(sd, fromNnf, txid);
     final File toFile = NNStorage.getStorageFile(sd, toNnf, txid);
     // renameTo fails on Windows if the destination file already exists.
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("renaming  " + fromFile.getAbsolutePath() 
-                + " to " + toFile.getAbsolutePath());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("renaming  {} to {}", fromFile.getAbsoluteFile(), toFile.getAbsolutePath());
     }
+
     if (!fromFile.renameTo(toFile)) {
       if (!toFile.delete() || !fromFile.renameTo(toFile)) {
         throw new IOException("renaming  " + fromFile.getAbsolutePath() + " to "  + 

+ 16 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1407,8 +1407,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         }
 
         if (LOG.isDebugEnabled()) {
-          LOG.debug("NameNode metadata after re-processing " +
-              "replication and invalidation queues during failover:\n" +
+          LOG.debug("NameNode metadata after re-processing {}"
+              + "replication and invalidation queues during failover:\n",
               metaSaveAsString());
         }
 
@@ -2675,8 +2675,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     for (CryptoProtocolVersion c : supportedVersions) {
       if (c.equals(CryptoProtocolVersion.UNKNOWN)) {
-        LOG.debug("Ignoring unknown CryptoProtocolVersion provided by " +
-            "client: {}", c.getUnknownValue());
+        LOG.debug("Ignoring unknown CryptoProtocolVersion provided by client: {}",
+            c.getUnknownValue());
         continue;
       }
       if (c.equals(required)) {
@@ -2987,8 +2987,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       requireEffectiveLayoutVersionForFeature(Feature.APPEND_NEW_BLOCK);
     }
 
-    NameNode.stateChangeLog.debug(
-        "DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}",
+    NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}",
         srcArg, holder, clientMachine);
     try {
       boolean skipSync = false;
@@ -3045,8 +3044,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       DatanodeInfo[] excludedNodes, String[] favoredNodes,
       EnumSet<AddBlockFlag> flags) throws IOException {
     final String operationName = "getAdditionalBlock";
-    NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {}  inodeId {}" +
-        " for {}", src, fileId, clientName);
+    NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {}  inodeId {} for {}",
+        src, fileId, clientName);
 
     LocatedBlock[] onRetryBlock = new LocatedBlock[1];
     FSDirWriteFileOp.ValidateAddBlockResult r;
@@ -3148,8 +3147,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
       throws IOException {
-    NameNode.stateChangeLog.debug(
-        "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
+    NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(null);
@@ -3158,8 +3156,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
       FSDirWriteFileOp.abandonBlock(dir, pc, b, fileId, src, holder);
-      NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: {} is " +
-          "removed from pendingCreates", b);
+      NameNode.stateChangeLog.debug(
+          "BLOCK* NameSystem.abandonBlock: {} is removed from pendingCreates", b);
     } finally {
       writeUnlock("abandonBlock");
     }
@@ -4016,9 +4014,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if ((!iFile.isUnderConstruction() || storedBlock.isComplete()) &&
           iFile.getLastBlock().isComplete()) {
         if (LOG.isDebugEnabled()) {
-          LOG.debug("Unexpected block (={}) since the file (={}) is not "
-                + "under construction", oldBlock, iFile.getLocalName());
+          LOG.debug("Unexpected block (={}) since the file (={}) is not under construction",
+              oldBlock, iFile.getLocalName());
         }
+
         return;
       }
 
@@ -4478,8 +4477,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     // file is closed
     getEditLog().logCloseFile(path, file);
-    NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted" +
-        " to the file system", path, file.getBlocks().length);
+    NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted to the file system",
+        path, file.getBlocks().length);
   }
 
   /**
@@ -6107,9 +6106,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (cookieTab[0] == null) {
         cookieTab[0] = String.valueOf(getIntCookie(cookieTab[0]));
       }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("there are no corrupt file blocks.");
-      }
+      LOG.debug("there are no corrupt file blocks.");
       return corruptFiles;
     }
 

+ 29 - 55
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -810,10 +810,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       throws IOException {
     checkNNStartup();
     String clientMachine = getClientMachine();
-    if (stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* NameNode.create: file "
-          +src+" for "+clientName+" at "+clientMachine);
-    }
+    stateChangeLog.debug("*DIR* NameNode.create: file {} for {} at {}.",
+        src, clientName, clientMachine);
     if (!checkPathLength(src)) {
       throw new IOException("create: Pathname too long.  Limit "
           + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@@ -845,10 +843,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       EnumSetWritable<CreateFlag> flag) throws IOException {
     checkNNStartup();
     String clientMachine = getClientMachine();
-    if (stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* NameNode.append: file "
-          +src+" for "+clientName+" at "+clientMachine);
-    }
+    stateChangeLog.debug("*DIR* NameNode.append: file {} for {} at {}.",
+        src, clientName, clientMachine);
     namesystem.checkOperation(OperationCategory.WRITE);
     CacheEntryWithPayload cacheEntry = getCacheEntryWithPayload(null);
     if (cacheEntry != null && cacheEntry.isSuccess()) {
@@ -894,8 +890,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
     checkNNStartup();
-    stateChangeLog.debug("*DIR* NameNode.setStoragePolicy for path: {}, " +
-            "policyName: {}", src, policyName);
+    stateChangeLog.debug("*DIR* NameNode.setStoragePolicy for path: {}, policyName: {}",
+        src, policyName);
     namesystem.setStoragePolicy(src, policyName);
   }
 
@@ -949,13 +945,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       ) throws IOException {
     checkNNStartup();
     if (LOG.isDebugEnabled()) {
-      LOG.debug("getAdditionalDatanode: src=" + src
-          + ", fileId=" + fileId
-          + ", blk=" + blk
-          + ", existings=" + Arrays.asList(existings)
-          + ", excludes=" + Arrays.asList(excludes)
-          + ", numAdditionalNodes=" + numAdditionalNodes
-          + ", clientName=" + clientName);
+      LOG.debug("getAdditionalDatanode: src={}, fileId={}, blk={}, existings={}, excludes={}"
+          + ", numAdditionalNodes={}, clientName={}", src, fileId, blk, Arrays.asList(existings),
+          Arrays.asList(excludes), numAdditionalNodes, clientName);
     }
 
     metrics.incrGetAdditionalDatanodeOps();
@@ -1053,9 +1045,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public boolean rename(String src, String dst) throws IOException {
     checkNNStartup();
-    if(stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
-    }
+    stateChangeLog.debug("*DIR* NameNode.rename: {} to {}.", src, dst);
     if (!checkPathLength(dst)) {
       throw new IOException("rename: Pathname too long.  Limit "
           + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@@ -1081,8 +1071,10 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public void concat(String trg, String[] src) throws IOException {
     checkNNStartup();
-    stateChangeLog.debug("*DIR* NameNode.concat: src path {} to" +
-        " target path {}", Arrays.toString(src), trg);
+    if (stateChangeLog.isDebugEnabled()) {
+      stateChangeLog.debug("*DIR* NameNode.concat: src path {} to target path {}",
+          Arrays.toString(src), trg);
+    }
     namesystem.checkOperation(OperationCategory.WRITE);
     CacheEntry cacheEntry = getCacheEntry();
     if (cacheEntry != null && cacheEntry.isSuccess()) {
@@ -1102,9 +1094,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public void rename2(String src, String dst, Options.Rename... options)
       throws IOException {
     checkNNStartup();
-    if(stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst);
-    }
+    stateChangeLog.debug("*DIR* NameNode.rename: {} to {}.", src, dst);
     if (!checkPathLength(dst)) {
       throw new IOException("rename: Pathname too long.  Limit "
           + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@@ -1128,8 +1118,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public boolean truncate(String src, long newLength, String clientName)
       throws IOException {
     checkNNStartup();
-    stateChangeLog
-        .debug("*DIR* NameNode.truncate: " + src + " to " + newLength);
+    stateChangeLog.debug("*DIR* NameNode.truncate: {} to {}", src, newLength);
     String clientMachine = getClientMachine();
     try {
       return namesystem.truncate(
@@ -1142,10 +1131,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   @Override // ClientProtocol
   public boolean delete(String src, boolean recursive) throws IOException {
     checkNNStartup();
-    if (stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* Namenode.delete: src=" + src
-          + ", recursive=" + recursive);
-    }
+    stateChangeLog.debug("*DIR* Namenode.delete: src={}, recursive={}.", src, recursive);
     namesystem.checkOperation(OperationCategory.WRITE);
     CacheEntry cacheEntry = getCacheEntry();
     if (cacheEntry != null && cacheEntry.isSuccess()) {
@@ -1178,9 +1164,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
       throws IOException {
     checkNNStartup();
-    if(stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src);
-    }
+    stateChangeLog.debug("*DIR* NameNode.mkdirs: {}.", src);
     if (!checkPathLength(src)) {
       throw new IOException("mkdirs: Pathname too long.  Limit " 
                             + MAX_PATH_LENGTH + " characters, " + MAX_PATH_DEPTH + " levels.");
@@ -1649,10 +1633,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
         final BlockReportContext context) throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
-    if(blockStateChangeLog.isDebugEnabled()) {
-      blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
-           + "from " + nodeReg + ", reports.length=" + reports.length);
-    }
+    blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: from {}, reports.length={}.",
+        nodeReg, reports.length);
     final BlockManager bm = namesystem.getBlockManager(); 
     boolean noStaleStorages = false;
     try {
@@ -1695,10 +1677,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
       String poolId, List<Long> blockIds) throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
-    if (blockStateChangeLog.isDebugEnabled()) {
-      blockStateChangeLog.debug("*BLOCK* NameNode.cacheReport: "
-           + "from " + nodeReg + " " + blockIds.size() + " blocks");
-    }
+    blockStateChangeLog.debug("*BLOCK* NameNode.cacheReport: from {} {} blocks",
+        nodeReg, blockIds.size());
     namesystem.getCacheManager().processCacheReport(nodeReg, blockIds);
     return null;
   }
@@ -1710,11 +1690,8 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     checkNNStartup();
     verifyRequest(nodeReg);
     metrics.incrBlockReceivedAndDeletedOps();
-    if(blockStateChangeLog.isDebugEnabled()) {
-      blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: "
-          +"from "+nodeReg+" "+receivedAndDeletedBlocks.length
-          +" blocks.");
-    }
+    blockStateChangeLog.debug("*BLOCK* NameNode.blockReceivedAndDeleted: from {} {} blocks.",
+        nodeReg, receivedAndDeletedBlocks.length);
     final BlockManager bm = namesystem.getBlockManager();
     for (final StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) {
       bm.enqueueBlockOp(new Runnable() {
@@ -1842,9 +1819,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   
   @Override // GetUserMappingsProtocol
   public String[] getGroupsForUser(String user) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Getting groups for user " + user);
-    }
+    LOG.debug("Getting groups for user {}", user);
     return UserGroupInformation.createRemoteUser(user).getGroupNames();
   }
 
@@ -2022,9 +1997,9 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public void renameSnapshot(String snapshotRoot, String snapshotOldName,
       String snapshotNewName) throws IOException {
     checkNNStartup();
-    LOG.debug("*DIR* NameNode.renameSnapshot: Snapshot Path {}, " +
-        "snapshotOldName {}, snapshotNewName {}", snapshotRoot,
-        snapshotOldName, snapshotNewName);
+    LOG.debug(
+        "*DIR* NameNode.renameSnapshot: Snapshot Path {},snapshotOldName {}, snapshotNewName {}",
+        snapshotRoot, snapshotOldName, snapshotNewName);
     if (snapshotNewName == null || snapshotNewName.isEmpty()) {
       throw new IOException("The new snapshot name is null or empty.");
     }
@@ -2318,8 +2293,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
     try {
       if (ecPolicyName == null) {
         ecPolicyName = defaultECPolicyName;
-        LOG.debug("No policy name is specified, " +
-            "set the default policy name instead");
+        LOG.debug("No policy name is specified, set the default policy name instead");
       }
       LOG.debug("Set erasure coding policy {} on {}", ecPolicyName, src);
       namesystem.setErasureCodingPolicy(src, ecPolicyName, cacheEntry != null);

+ 3 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java

@@ -262,9 +262,7 @@ public class EditLogTailer {
     nnCount = nns.size();
     // setup the iterator to endlessly loop the nns
     this.nnLookup = Iterators.cycle(nns);
-
-    LOG.debug("logRollPeriodMs=" + logRollPeriodMs +
-        " sleepTime=" + sleepTimeMs);
+    LOG.debug("logRollPeriodMs={} sleepTime={}.", logRollPeriodMs, sleepTimeMs);
   }
 
   public void start() {
@@ -360,9 +358,7 @@ public class EditLogTailer {
             currentLastTxnId, lastTxnId);
         return 0;
       }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("edit streams to load from: " + streams.size());
-      }
+      LOG.debug("edit streams to load from: {}.", streams.size());
       
       // Once we have streams to load, errors encountered are legitimate cause
       // for concern, so we don't catch them here. Simple errors reading from
@@ -375,10 +371,7 @@ public class EditLogTailer {
         editsLoaded = elie.getNumEditsLoaded();
         throw elie;
       } finally {
-        if (editsLoaded > 0 || LOG.isDebugEnabled()) {
-          LOG.debug(String.format("Loaded %d edits starting from txid %d ",
-              editsLoaded, lastTxnId));
-        }
+        LOG.debug("Loaded {} edits starting from txid {}.", editsLoaded, lastTxnId);
         NameNode.getNameNodeMetrics().addNumEditLogLoaded(editsLoaded);
       }