Quellcode durchsuchen

HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client package. Contributed by Mingliang Liu.

Haohui Mai vor 10 Jahren
Ursprung
Commit
39285e6a19
24 geänderte Dateien mit 230 neuen und 573 gelöschten Zeilen
  1. 35 79
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  2. 25 52
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  3. 6 14
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
  4. 11 25
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  5. 28 57
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  6. 13 23
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  7. 2 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
  8. 13 34
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  9. 1 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  10. 7 18
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  11. 4 14
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
  12. 3 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
  13. 2 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  14. 3 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
  15. 3 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  16. 13 32
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
  17. 23 78
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
  18. 9 24
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
  19. 4 10
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
  20. 12 51
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
  21. 2 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
  22. 1 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
  23. 7 21
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  24. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

+ 35 - 79
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -349,17 +349,13 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       if (clientContext.getUseLegacyBlockReaderLocal()) {
         reader = getLegacyBlockReaderLocal();
         if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new legacy block reader local.");
-          }
+          LOG.trace("{}: returning new legacy block reader local.", this);
           return reader;
         }
       } else {
         reader = getBlockReaderLocal();
         if (reader != null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": returning new block reader local.");
-          }
+          LOG.trace("{}: returning new block reader local.", this);
           return reader;
         }
       }
@@ -367,10 +363,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (scConf.isDomainSocketDataTraffic()) {
       reader = getRemoteBlockReaderFromDomain();
       if (reader != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": returning new remote block reader using " +
-              "UNIX domain socket on " + pathInfo.getPath());
-        }
+        LOG.trace("{}: returning new remote block reader using UNIX domain "
+            + "socket on {}", this, pathInfo.getPath());
         return reader;
       }
     }
@@ -405,10 +399,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             setVisibleLength(visibleLength).
             build();
         if (accessor == null) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": No ReplicaAccessor created by " +
-                cls.getName());
-          }
+          LOG.trace("{}: No ReplicaAccessor created by {}",
+              this, cls.getName());
         } else {
           return new ExternalBlockReader(accessor, visibleLength, startOffset);
         }
@@ -427,14 +419,10 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * first introduced in HDFS-2246.
    */
   private BlockReader getLegacyBlockReaderLocal() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
-    }
+    LOG.trace("{}: trying to construct BlockReaderLocalLegacy", this);
     if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
-            "the address " + inetSocketAddress + " is not local");
-      }
+      LOG.trace("{}: can't construct BlockReaderLocalLegacy because the address"
+          + "{} is not local", this, inetSocketAddress);
       return null;
     }
     if (clientContext.getDisableLegacyBlockReaderLocal()) {
@@ -470,10 +458,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   }
 
   private BlockReader getBlockReaderLocal() throws InvalidToken {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to construct a BlockReaderLocal " +
-          "for short-circuit reads.");
-    }
+    LOG.trace("{}: trying to construct a BlockReaderLocal for short-circuit "
+        + " reads.", this);
     if (pathInfo == null) {
       pathInfo = clientContext.getDomainSocketFactory()
           .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
@@ -488,10 +474,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
     InvalidToken exc = info.getInvalidTokenException();
     if (exc != null) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": got InvalidToken exception while trying to " +
-            "construct BlockReaderLocal via " + pathInfo.getPath());
-      }
+      LOG.trace("{}: got InvalidToken exception while trying to construct "
+          + "BlockReaderLocal via {}", this, pathInfo.getPath());
       throw exc;
     }
     if (info.getReplica() == null) {
@@ -527,9 +511,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
       if (info != null) return info;
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
-    }
+    LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
     BlockReaderPeer curPeer;
     while (true) {
       curPeer = nextDomainPeer();
@@ -544,10 +526,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
             clientName);
         if (usedPeer.booleanValue()) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": allocShmSlot used up our previous socket " +
-              peer.getDomainSocket() + ".  Allocating a new one...");
-          }
+          LOG.trace("{}: allocShmSlot used up our previous socket {}.  "
+              + "Allocating a new one...", this, peer.getDomainSocket());
           curPeer = nextDomainPeer();
           if (curPeer == null) break;
           peer = (DomainPeer)curPeer.peer;
@@ -562,9 +542,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         if (curPeer.fromCache) {
           // Handle an I/O error we got when using a cached socket.
           // These are considered less serious, because the socket may be stale.
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(this + ": closing stale domain peer " + peer, e);
-          }
+          LOG.debug("{}: closing stale domain peer {}", this, peer, e);
           IOUtilsClient.cleanup(LOG, peer);
         } else {
           // Handle an I/O error we got when using a newly created socket.
@@ -617,7 +595,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         ExtendedBlockId key =
             new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
         if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
-          LOG.trace("Sending receipt verification byte for slot " + slot);
+          LOG.trace("Sending receipt verification byte for slot {}", slot);
           sock.getOutputStream().write(0);
         }
         replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
@@ -650,9 +628,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       String msg = "access control error while " +
           "attempting to set up short-circuit access to " +
           fileName + resp.getMessage();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(this + ":" + msg);
-      }
+      LOG.debug("{}:{}", this, msg);
       return new ShortCircuitReplicaInfo(new InvalidToken(msg));
     default:
       LOG.warn(this + ": unknown response code " + resp.getStatus() +
@@ -684,10 +660,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
            " is not usable.", this, pathInfo);
       return null;
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create a remote block reader from the " +
-          "UNIX domain socket at " + pathInfo.getPath());
-    }
+    LOG.trace("{}: trying to create a remote block reader from the UNIX domain "
+        + "socket at {}", this, pathInfo.getPath());
 
     while (true) {
       BlockReaderPeer curPeer = nextDomainPeer();
@@ -701,19 +675,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       } catch (IOException ioe) {
         IOUtilsClient.cleanup(LOG, peer);
         if (isSecurityException(ioe)) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": got security exception while constructing " +
-                "a remote block reader from the unix domain socket at " +
-                pathInfo.getPath(), ioe);
-          }
+          LOG.trace("{}: got security exception while constructing a remote "
+                  + " block reader from the unix domain socket at {}",
+              this, pathInfo.getPath(), ioe);
           throw ioe;
         }
         if (curPeer.fromCache) {
           // Handle an I/O error we got when using a cached peer.  These are
           // considered less serious, because the underlying socket may be stale.
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Closed potentially stale domain peer " + peer, ioe);
-          }
+          LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
         } else {
           // Handle an I/O error we got when using a newly created domain peer.
           // We temporarily disable the domain socket path for a few minutes in
@@ -747,10 +717,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    *             If there was another problem.
    */
   private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": trying to create a remote block reader from a " +
-          "TCP socket");
-    }
+    LOG.trace("{}: trying to create a remote block reader from a TCP socket",
+        this);
     BlockReader blockReader = null;
     while (true) {
       BlockReaderPeer curPeer = null;
@@ -763,19 +731,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         return blockReader;
       } catch (IOException ioe) {
         if (isSecurityException(ioe)) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": got security exception while constructing " +
-                "a remote block reader from " + peer, ioe);
-          }
+          LOG.trace("{}: got security exception while constructing a remote "
+              + "block reader from {}", this, peer, ioe);
           throw ioe;
         }
         if ((curPeer != null) && curPeer.fromCache) {
           // Handle an I/O error we got when using a cached peer.  These are
           // considered less serious, because the underlying socket may be
           // stale.
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Closed potentially stale remote peer " + peer, ioe);
-          }
+          LOG.debug("Closed potentially stale remote peer {}", peer, ioe);
         } else {
           // Handle an I/O error we got when using a newly created peer.
           LOG.warn("I/O error constructing remote block reader.", ioe);
@@ -808,9 +772,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, true);
       if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextDomainPeer: reusing existing peer " + peer);
-        }
+        LOG.trace("nextDomainPeer: reusing existing peer {}", peer);
         return new BlockReaderPeer(peer, true);
       }
     }
@@ -832,24 +794,18 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     if (remainingCacheTries > 0) {
       Peer peer = clientContext.getPeerCache().get(datanode, false);
       if (peer != null) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("nextTcpPeer: reusing existing peer " + peer);
-        }
+        LOG.trace("nextTcpPeer: reusing existing peer {}", peer);
         return new BlockReaderPeer(peer, true);
       }
     }
     try {
       Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
         datanode);
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("nextTcpPeer: created newConnectedPeer " + peer);
-      }
+      LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
       return new BlockReaderPeer(peer, false);
     } catch (IOException e) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("nextTcpPeer: failed to create newConnectedPeer " +
-                  "connected to " + datanode);
-      }
+      LOG.trace("nextTcpPeer: failed to create newConnectedPeer connected to"
+          + "{}", datanode);
       throw e;
     }
   }

+ 25 - 52
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -412,17 +412,10 @@ class BlockReaderLocal implements BlockReader {
   public synchronized int read(ByteBuffer buf) throws IOException {
     boolean canSkipChecksum = createNoChecksumContext();
     try {
-      String traceString = null;
-      if (LOG.isTraceEnabled()) {
-        traceString = new StringBuilder().
-            append("read(").
-            append("buf.remaining=").append(buf.remaining()).
-            append(", block=").append(block).
-            append(", filename=").append(filename).
-            append(", canSkipChecksum=").append(canSkipChecksum).
-            append(")").toString();
-        LOG.info(traceString + ": starting");
-      }
+      String traceFormatStr = "read(buf.remaining={}, block={}, filename={}, "
+          + "canSkipChecksum={})";
+      LOG.trace(traceFormatStr + ": starting",
+          buf.remaining(), block, filename, canSkipChecksum);
       int nRead;
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
@@ -431,14 +424,12 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(buf, canSkipChecksum);
         }
       } catch (IOException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.info(traceString + ": I/O error", e);
-        }
+        LOG.trace(traceFormatStr + ": I/O error",
+            buf.remaining(), block, filename, canSkipChecksum, e);
         throw e;
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.info(traceString + ": returning " + nRead);
-      }
+      LOG.trace(traceFormatStr + ": returning {}",
+          buf.remaining(), block, filename, canSkipChecksum, nRead);
       return nRead;
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
@@ -490,10 +481,8 @@ class BlockReaderLocal implements BlockReader {
     }
     dataBuf.limit(dataBuf.position());
     dataBuf.position(Math.min(dataBuf.position(), slop));
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("loaded " + dataBuf.remaining() + " bytes into bounce " +
-          "buffer from offset " + oldDataPos + " of " + block);
-    }
+    LOG.trace("loaded {} bytes into bounce buffer from offset {} of {}",
+        dataBuf.remaining(), oldDataPos, block);
     return dataBuf.limit() != maxReadaheadLength;
   }
 
@@ -565,18 +554,10 @@ class BlockReaderLocal implements BlockReader {
     boolean canSkipChecksum = createNoChecksumContext();
     int nRead;
     try {
-      String traceString = null;
-      if (LOG.isTraceEnabled()) {
-        traceString = new StringBuilder().
-            append("read(arr.length=").append(arr.length).
-            append(", off=").append(off).
-            append(", len=").append(len).
-            append(", filename=").append(filename).
-            append(", block=").append(block).
-            append(", canSkipChecksum=").append(canSkipChecksum).
-            append(")").toString();
-        LOG.trace(traceString + ": starting");
-      }
+      final String traceFormatStr = "read(arr.length={}, off={}, len={}, "
+          + "filename={}, block={}, canSkipChecksum={})";
+      LOG.trace(traceFormatStr + ": starting",
+          arr.length, off, len, filename, block, canSkipChecksum);
       try {
         if (canSkipChecksum && zeroReadaheadRequested) {
           nRead = readWithoutBounceBuffer(arr, off, len);
@@ -584,14 +565,12 @@ class BlockReaderLocal implements BlockReader {
           nRead = readWithBounceBuffer(arr, off, len, canSkipChecksum);
         }
       } catch (IOException e) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(traceString + ": I/O error", e);
-        }
+        LOG.trace(traceFormatStr + ": I/O error",
+            arr.length, off, len, filename, block, canSkipChecksum, e);
         throw e;
       }
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(traceString + ": returning " + nRead);
-      }
+      LOG.trace(traceFormatStr + ": returning {}",
+          arr.length, off, len, filename, block, canSkipChecksum, nRead);
     } finally {
       if (canSkipChecksum) releaseNoChecksumContext();
     }
@@ -634,11 +613,9 @@ class BlockReaderLocal implements BlockReader {
       dataBuf.position(dataBuf.position() + discardedFromBuf);
       remaining -= discardedFromBuf;
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("skip(n=" + n + ", block=" + block + ", filename=" + 
-        filename + "): discarded " + discardedFromBuf + " bytes from " +
-        "dataBuf and advanced dataPos by " + remaining);
-    }
+    LOG.trace("skip(n={}, block={}, filename={}): discarded {} bytes from "
+            + "dataBuf and advanced dataPos by {}",
+        n, block, filename, discardedFromBuf, remaining);
     dataPos += remaining;
     return n;
   }
@@ -653,9 +630,7 @@ class BlockReaderLocal implements BlockReader {
   public synchronized void close() throws IOException {
     if (closed) return;
     closed = true;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("close(filename=" + filename + ", block=" + block + ")");
-    }
+    LOG.trace("close(filename={}, block={})", filename, block);
     replica.unref();
     freeDataBufIfExists();
     freeChecksumBufIfExists();
@@ -705,11 +680,9 @@ class BlockReaderLocal implements BlockReader {
         (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
     if (anchor) {
       if (!createNoChecksumContext()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("can't get an mmap for " + block + " of " + filename + 
-              " since SKIP_CHECKSUMS was not given, " +
-              "we aren't skipping checksums, and the block is not mlocked.");
-        }
+        LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
+            + "given, we aren't skipping checksums, and the block is not "
+            + "mlocked.", block, filename);
         return null;
       }
     }

+ 6 - 14
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java

@@ -221,11 +221,9 @@ class BlockReaderLocalLegacy implements BlockReader {
       File blkfile = new File(pathinfo.getBlockPath());
       dataIn = new FileInputStream(blkfile);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("New BlockReaderLocalLegacy for file " + blkfile + " of size "
-            + blkfile.length() + " startOffset " + startOffset + " length "
-            + length + " short circuit checksum " + !skipChecksumCheck);
-      }
+      LOG.debug("New BlockReaderLocalLegacy for file {} of size {} startOffset "
+              + "{} length {} short circuit checksum {}",
+          blkfile, blkfile.length(), startOffset, length, !skipChecksumCheck);
 
       if (!skipChecksumCheck) {
         // get the metadata file
@@ -292,9 +290,7 @@ class BlockReaderLocalLegacy implements BlockReader {
       // channel for the DataNode to notify the client that the path has been
       // invalidated.  Therefore, our only option is to skip caching.
       if (pathinfo != null && !storageType.isTransient()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Cached location of block " + blk + " as " + pathinfo);
-        }
+        LOG.debug("Cached location of block {} as {}", blk, pathinfo);
         localDatanodeInfo.setBlockLocalPathInfo(blk, pathinfo);
       }
     } catch (IOException e) {
@@ -603,9 +599,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized int read(byte[] buf, int off, int len) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("read off " + off + " len " + len);
-    }
+    LOG.trace("read off {} len {}", off, len);
     if (!verifyChecksum) {
       return dataIn.read(buf, off, len);
     }
@@ -624,9 +618,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
   @Override
   public synchronized long skip(long n) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("skip " + n);
-    }
+    LOG.debug("skip {}", n);
     if (n <= 0) {
       return 0;
     }

+ 11 - 25
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -419,9 +419,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     final int idx = r.nextInt(localInterfaceAddrs.length);
     final SocketAddress addr = localInterfaceAddrs[idx];
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using local interface " + addr);
-    }
+    LOG.debug("Using local interface {}", addr);
     return addr;
   }
 
@@ -1216,9 +1214,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                              InetSocketAddress[] favoredNodes) throws IOException {
     checkOpen();
     final FsPermission masked = applyUMask(permission);
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + masked);
-    }
+    LOG.debug("{}: masked={}", src, masked);
     final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
         src, masked, flag, createParent, replication, blockSize, progress,
         buffersize, dfsClientConf.createChecksum(checksumOpt),
@@ -1815,10 +1811,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
               smallBufferSize));
           in = new DataInputStream(pair.in);
 
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("write to " + datanodes[j] + ": "
-                + Op.BLOCK_CHECKSUM + ", block=" + block);
-          }
+          LOG.debug("write to {}: {}, block={}",
+              datanodes[j], Op.BLOCK_CHECKSUM, block);
           // get block MD5
           new Sender(out).blockChecksum(block, lb.getBlockToken());
 
@@ -1882,12 +1876,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
           }
         } catch (InvalidBlockTokenException ibte) {
           if (i > lastRetriedIndex) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
-                  + "for file " + src + " for block " + block
-                  + " from datanode " + datanodes[j]
-                  + ". Will retry the block once.");
-            }
+            LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
+                  + "for file {} for block {} from datanode {}. Will retry the "
+                  + "block once.",
+                src, block, datanodes[j]);
             lastRetriedIndex = i;
             done = true; // actually it's not done; but we'll retry
             i--; // repeat at i-th block
@@ -1941,9 +1933,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     try {
       sock = socketFactory.createSocket();
       String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Connecting to datanode " + dnAddr);
-      }
+      LOG.debug("Connecting to datanode {}", dnAddr);
       NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
       sock.setSoTimeout(timeout);
   
@@ -2563,9 +2553,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       absPermission = applyUMask(null);
     } 
 
-    if(LOG.isDebugEnabled()) {
-      LOG.debug(src + ": masked=" + absPermission);
-    }
+    LOG.debug("{}: masked={}", src, absPermission);
     TraceScope scope = tracer.newScope("mkdir");
     try {
       return namenode.mkdirs(src, absPermission, createParent);
@@ -3061,9 +3049,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       }
     });
     HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using hedged reads; pool threads=" + num);
-    }
+    LOG.debug("Using hedged reads; pool threads={}", num);
   }
 
   ThreadPoolExecutor getHedgedReadsThreadPool() {

+ 28 - 57
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -315,9 +315,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if (locatedBlocks == null || refresh) {
       newInfo = dfsClient.getLocatedBlocks(src, 0);
     }
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("newInfo = " + newInfo);
-    }
+    DFSClient.LOG.debug("newInfo = {}", newInfo);
     if (newInfo == null) {
       throw new IOException("Cannot open filename " + src);
     }
@@ -383,10 +381,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           replicaNotFoundCount--;
         }
         
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
-              + datanode + " for block " + locatedblock.getBlock(), ioe);
-        }
+        DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
+              + " for block {}", datanode, locatedblock.getBlock(), ioe);
       } finally {
         if (cdp != null) {
           RPC.stopProxy(cdp);
@@ -1067,9 +1063,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     final String dnAddr =
         chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
-    }
+    DFSClient.LOG.debug("Connecting to datanode {}", dnAddr);
     InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
     return new DNAddrPair(chosenNode, targetAddr, storageType);
   }
@@ -1309,11 +1303,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             future.get();
             return;
           }
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
-                + "ms to read from " + chosenNode.info
-                + "; spawning hedged read");
-          }
+          DFSClient.LOG.debug("Waited {}ms to read from {}; spawning hedged "
+              + "read", conf.getHedgedReadThresholdMillis(), chosenNode.info);
           // Ignore this node on next go around.
           ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
@@ -1340,10 +1331,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
               .submit(getFromDataNodeCallable);
           futures.add(oneMoreRequest);
         } catch (IOException ioe) {
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Failed getting node for hedged read: "
-                + ioe.getMessage());
-          }
+          DFSClient.LOG.debug("Failed getting node for hedged read: {}",
+              ioe.getMessage());
         }
         // if not succeeded. Submit callables for each datanode in a loop, wait
         // for a fixed interval and get the result from the fastest one.
@@ -1599,11 +1588,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             throw new IOException(errMsg);
           }
         } catch (IOException e) {//make following read to retry
-          if(DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("Exception while seek to " + targetPos
-                + " from " + getCurrentBlock() + " of " + src + " from "
-                + currentNode, e);
-          }
+          DFSClient.LOG.debug("Exception while seek to {} from {} of {} from "
+              + "{}", targetPos, getCurrentBlock(), src, currentNode, e);
         }
       }
     }
@@ -1819,20 +1805,16 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     } else {
       length63 = 1 + curEnd - curPos;
       if (length63 <= 0) {
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
-            curPos + " of " + src + "; " + length63 + " bytes left in block.  " +
-            "blockPos=" + blockPos + "; curPos=" + curPos +
-            "; curEnd=" + curEnd);
-        }
+        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {}"
+                + " of {}; {} bytes left in block. blockPos={}; curPos={};"
+                + "curEnd={}",
+            curPos, src, length63, blockPos, curPos, curEnd);
         return null;
       }
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Reducing read length from " + maxLength +
-            " to " + length63 + " to avoid going more than one byte " +
-            "past the end of the block.  blockPos=" + blockPos +
-            "; curPos=" + curPos + "; curEnd=" + curEnd);
-      }
+      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid going "
+              + "more than one byte past the end of the block.  blockPos={}; "
+              +" curPos={}; curEnd={}",
+          maxLength, length63, blockPos, curPos, curEnd);
     }
     // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
     int length;
@@ -1846,28 +1828,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         // So we can't mmap the parts of the block higher than the 2 GB offset.
         // FIXME: we could work around this with multiple memory maps.
         // See HDFS-5101.
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
-            curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
-            "exceeded.  blockPos=" + blockPos + ", curEnd=" + curEnd);
-        }
+        DFSClient.LOG.debug("Unable to perform a zero-copy read from offset {} "
+            + " of {}; 31-bit MappedByteBuffer limit exceeded.  blockPos={}, "
+            + "curEnd={}", curPos, src, blockPos, curEnd);
         return null;
       }
       length = (int)length31;
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Reducing read length from " + maxLength +
-            " to " + length + " to avoid 31-bit limit.  " +
-            "blockPos=" + blockPos + "; curPos=" + curPos +
-            "; curEnd=" + curEnd);
-      }
+      DFSClient.LOG.debug("Reducing read length from {} to {} to avoid 31-bit "
+          + "limit.  blockPos={}; curPos={}; curEnd={}",
+          maxLength, length, blockPos, curPos, curEnd);
     }
     final ClientMmap clientMmap = blockReader.getClientMmap(opts);
     if (clientMmap == null) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
-          curPos + " of " + src + "; BlockReader#getClientMmap returned " +
-          "null.");
-      }
+      DFSClient.LOG.debug("unable to perform a zero-copy read from offset {} of"
+          + " {}; BlockReader#getClientMmap returned null.", curPos, src);
       return null;
     }
     boolean success = false;
@@ -1881,11 +1855,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       synchronized (infoLock) {
         readStatistics.addZeroCopyBytes(length);
       }
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("readZeroCopy read " + length + 
-            " bytes from offset " + curPos + " via the zero-copy read " +
-            "path.  blockEnd = " + blockEnd);
-      }
+      DFSClient.LOG.debug("readZeroCopy read {} bytes from offset {} via the "
+          + "zero-copy read path.  blockEnd = {}", length, curPos, blockEnd);
       success = true;
     } finally {
       if (!success) {

+ 13 - 23
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -190,9 +190,9 @@ public class DFSOutputStream extends FSOutputSummer
     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
     this.cachingStrategy = new AtomicReference<CachingStrategy>(
         dfsClient.getDefaultWriteCachingStrategy());
-    if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug(
-          "Set non-null progress callback on DFSOutputStream " + src);
+    if (progress != null) {
+      DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
+          +"{}", src);
     }
     
     this.bytesPerChecksum = checksum.getBytesPerChecksum();
@@ -365,12 +365,9 @@ public class DFSOutputStream extends FSOutputSummer
     final int chunkSize = csize + getChecksumSize();
     chunksPerPacket = Math.max(bodySize/chunkSize, 1);
     packetSize = chunkSize*chunksPerPacket;
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("computePacketChunkSize: src=" + src +
-                ", chunkSize=" + chunkSize +
-                ", chunksPerPacket=" + chunksPerPacket +
-                ", packetSize=" + packetSize);
-    }
+    DFSClient.LOG.debug("computePacketChunkSize: src={}, chunkSize={}, "
+            + "chunksPerPacket={}, packetSize={}",
+        src, chunkSize, chunksPerPacket, packetSize);
   }
 
   protected TraceScope createWriteTraceScope() {
@@ -397,14 +394,10 @@ public class DFSOutputStream extends FSOutputSummer
     if (currentPacket == null) {
       currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
           .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + 
-            currentPacket.getSeqno() +
-            ", src=" + src +
-            ", packetSize=" + packetSize +
-            ", chunksPerPacket=" + chunksPerPacket +
-            ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
-      }
+      DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno={},"
+              + " src={}, packetSize={}, chunksPerPacket={}, bytesCurBlock={}",
+          currentPacket.getSeqno(), src, packetSize, chunksPerPacket,
+          getStreamer().getBytesCurBlock());
     }
 
     currentPacket.writeChecksum(checksum, ckoff, cklen);
@@ -558,12 +551,9 @@ public class DFSOutputStream extends FSOutputSummer
         int numKept = flushBuffer(!endBlock, true);
         // bytesCurBlock potentially incremented if there was buffered data
 
-        if (DFSClient.LOG.isDebugEnabled()) {
-          DFSClient.LOG.debug("DFSClient flush(): "
-              + " bytesCurBlock=" + getStreamer().getBytesCurBlock()
-              + " lastFlushOffset=" + lastFlushOffset
-              + " createNewBlock=" + endBlock);
-        }
+        DFSClient.LOG.debug("DFSClient flush():  bytesCurBlock={}, "
+                + "lastFlushOffset={}, createNewBlock={}",
+            getStreamer().getBytesCurBlock(), lastFlushOffset, endBlock);
         // Flush only if we haven't already flushed till this offset.
         if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
           assert getStreamer().getBytesCurBlock() > lastFlushOffset;

+ 2 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java

@@ -462,19 +462,13 @@ public class DFSUtilClient {
     InetAddress addr = targetAddr.getAddress();
     Boolean cached = localAddrMap.get(addr.getHostAddress());
     if (cached != null) {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Address " + targetAddr +
-            (cached ? " is local" : " is not local"));
-      }
+      LOG.trace("Address {} is {} local", targetAddr, (cached ? "" : "not"));
       return cached;
     }
 
     boolean local = NetUtils.isLocalAddress(addr);
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Address " + targetAddr +
-          (local ? " is local" : " is not local"));
-    }
+    LOG.trace("Address {} is {} local", targetAddr, (local ? "" : "not"));
     localAddrMap.put(addr.getHostAddress(), local);
     return local;
   }

+ 13 - 34
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

@@ -132,18 +132,14 @@ class DataStreamer extends Daemon {
       final int length, final DFSClient client) throws IOException {
     final DfsClientConf conf = client.getConf();
     final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr);
-    }
+    LOG.debug("Connecting to datanode {}", dnAddr);
     final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
     final Socket sock = client.socketFactory.createSocket();
     final int timeout = client.getDatanodeReadTimeout(length);
     NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
     sock.setSoTimeout(timeout);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Send buf size " + sock.getSendBufferSize());
-    }
+    LOG.debug("Send buf size {}", sock.getSendBufferSize());
     return sock;
   }
 
@@ -484,9 +480,7 @@ class DataStreamer extends Daemon {
   }
 
   private void endBlock() {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Closing old block " + block);
-    }
+    LOG.debug("Closing old block {}", block);
     this.setName("DataStreamer for file " + src);
     closeResponder();
     closeStream();
@@ -567,15 +561,11 @@ class DataStreamer extends Daemon {
 
         // get new block from namenode.
         if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Allocating new block");
-          }
+          LOG.debug("Allocating new block");
           setPipeline(nextBlockOutputStream());
           initDataStreaming();
         } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Append to block " + block);
-          }
+          LOG.debug("Append to block {}", block);
           setupPipelineForAppendOrRecovery();
           if (streamerClosed) {
             continue;
@@ -627,10 +617,7 @@ class DataStreamer extends Daemon {
           }
         }
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("DataStreamer block " + block +
-              " sending packet " + one);
-        }
+        LOG.debug("DataStreamer block {} sending packet {}", block, one);
 
         // write out data to remote datanode
         TraceScope writeScope = dfsClient.getTracer().
@@ -741,9 +728,7 @@ class DataStreamer extends Daemon {
     TraceScope scope = dfsClient.getTracer().
         newScope("waitForAckedSeqno");
     try {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Waiting for ack for: " + seqno);
-      }
+      LOG.debug("Waiting for ack for: {}", seqno);
       long begin = Time.monotonicNow();
       try {
         synchronized (dataQueue) {
@@ -955,8 +940,8 @@ class DataStreamer extends Daemon {
             LOG.warn("Slow ReadProcessor read fields took " + duration
                 + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
                 + ack + ", targets: " + Arrays.asList(targets));
-          } else if (LOG.isDebugEnabled()) {
-            LOG.debug("DFSClient " + ack);
+          } else {
+            LOG.debug("DFSClient {}", ack);
           }
 
           long seqno = ack.getSeqno();
@@ -1176,9 +1161,7 @@ class DataStreamer extends Daemon {
   }
 
   private void addDatanode2ExistingPipeline() throws IOException {
-    if (DataTransferProtocol.LOG.isDebugEnabled()) {
-      DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
-    }
+    DataTransferProtocol.LOG.debug("lastAckedSeqno = {}", lastAckedSeqno);
       /*
        * Is data transfer necessary?  We have the following cases.
        *
@@ -1645,10 +1628,8 @@ class DataStreamer extends Daemon {
           new HashSet<String>(Arrays.asList(favoredNodes));
       for (int i = 0; i < nodes.length; i++) {
         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(nodes[i].getXferAddrWithHostname() +
-              " was chosen by name node (favored=" + pinnings[i] + ").");
-        }
+        LOG.debug("{} was chosen by name node (favored={}).",
+            nodes[i].getXferAddrWithHostname(), pinnings[i]);
       }
       if (shouldLog && !favoredSet.isEmpty()) {
         // There is one or more favored nodes that were not allocated.
@@ -1787,9 +1768,7 @@ class DataStreamer extends Daemon {
       packet.addTraceParent(Tracer.getCurrentSpanId());
       dataQueue.addLast(packet);
       lastQueuedSeqno = packet.getSeqno();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Queued packet " + packet.getSeqno());
-      }
+      LOG.debug("Queued packet {}", packet.getSeqno());
       dataQueue.notifyAll();
     }
   }

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java

@@ -250,9 +250,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       PacketHeader header = new PacketHeader();
       header.readFields(in);
 
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("DFSClient readChunk got header " + header);
-      }
+      LOG.debug("DFSClient readChunk got header {}", header);
 
       // Sanity check the lengths
       if (!header.sanityCheck(lastSeqNo)) {

+ 7 - 18
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java

@@ -135,14 +135,9 @@ public class RemoteBlockReader2  implements BlockReader {
   @Override
   public synchronized int read(byte[] buf, int off, int len) 
                                throws IOException {
-
-    UUID randomId = null;
-    if (LOG.isTraceEnabled()) {
-      randomId = UUID.randomUUID();
-      LOG.trace(String.format("Starting read #%s file %s from datanode %s",
-        randomId.toString(), this.filename,
-        this.datanodeID.getHostName()));
-    }
+    UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
+    LOG.trace("Starting read #{} file {} from datanode {}",
+        randomId, filename, datanodeID.getHostName());
 
     if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
       TraceScope scope = tracer.newScope(
@@ -154,9 +149,7 @@ public class RemoteBlockReader2  implements BlockReader {
       }
     }
 
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("Finishing read #" + randomId));
-    }
+    LOG.trace("Finishing read #{}", randomId);
 
     if (curDataSlice.remaining() == 0) {
       // we're at EOF now
@@ -203,9 +196,7 @@ public class RemoteBlockReader2  implements BlockReader {
     curDataSlice = packetReceiver.getDataSlice();
     assert curDataSlice.capacity() == curHeader.getDataLen();
     
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("DFSClient readNextPacket got header " + curHeader);
-    }
+    LOG.trace("DFSClient readNextPacket got header {}", curHeader);
 
     // Sanity check the lengths
     if (!curHeader.sanityCheck(lastSeqNo)) {
@@ -276,10 +267,8 @@ public class RemoteBlockReader2  implements BlockReader {
   }
 
   private void readTrailingEmptyPacket() throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Reading empty packet at end of read");
-    }
-    
+    LOG.trace("Reading empty packet at end of read");
+
     packetReceiver.receiveNextPacket(in);
 
     PacketHeader trailer = packetReceiver.getHeader();

+ 4 - 14
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java

@@ -308,10 +308,7 @@ public class LeaseRenewer {
               }
               LeaseRenewer.this.run(id);
             } catch(InterruptedException e) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
-                    + " is interrupted.", e);
-              }
+              LOG.debug("LeaseRenewer is interrupted.", e);
             } finally {
               synchronized(LeaseRenewer.this) {
                 Factory.INSTANCE.remove(LeaseRenewer.this);
@@ -399,9 +396,7 @@ public class LeaseRenewer {
     }
 
     if (daemonCopy != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Wait for lease checker to terminate");
-      }
+      LOG.debug("Wait for lease checker to terminate");
       daemonCopy.join();
     }
   }
@@ -424,16 +419,11 @@ public class LeaseRenewer {
       //skip if current client name is the same as the previous name.
       if (!c.getClientName().equals(previousName)) {
         if (!c.renewLease()) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Did not renew lease for client " +
-                c);
-          }
+          LOG.debug("Did not renew lease for client {}", c);
           continue;
         }
         previousName = c.getClientName();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Lease renewed for client " + previousName);
-        }
+        LOG.debug("Lease renewed for client {}", previousName);
       }
     }
   }

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java

@@ -147,11 +147,9 @@ public class PacketReceiver implements Closeable {
       throw new IOException("Invalid header length " + headerLen);
     }
     
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("readNextPacket: dataPlusChecksumLen = " + dataPlusChecksumLen +
-          " headerLen = " + headerLen);
-    }
-    
+    LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
+        dataPlusChecksumLen, headerLen);
+
     // Sanity check the buffer size so we don't allocate too much memory
     // and OOME.
     int totalLen = payloadLen + headerLen;

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java

@@ -73,10 +73,8 @@ public class Sender implements DataTransferProtocol {
 
   private static void send(final DataOutputStream out, final Op opcode,
       final Message proto) throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Sending DataTransferOp " + proto.getClass().getSimpleName()
-          + ": " + proto);
-    }
+    LOG.trace("Sending DataTransferOp {}: {}",
+        proto.getClass().getSimpleName(), proto);
     op(out, opcode);
     proto.writeDelimitedTo(out);
     out.flush();

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java

@@ -332,11 +332,9 @@ public final class DataTransferSaslUtil {
   public static IOStreamPair createStreamPair(Configuration conf,
       CipherOption cipherOption, OutputStream out, InputStream in, 
       boolean isServer) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Creating IOStreamPair of CryptoInputStream and " +
-          "CryptoOutputStream.");
-    }
-    CryptoCodec codec = CryptoCodec.getInstance(conf, 
+    LOG.debug("Creating IOStreamPair of CryptoInputStream and "
+        + "CryptoOutputStream.");
+    CryptoCodec codec = CryptoCodec.getInstance(conf,
         cipherOption.getCipherSuite());
     byte[] inKey = cipherOption.getInKey();
     byte[] inIv = cipherOption.getInIv();

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -130,9 +130,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
       throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
-    }
+    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
     rpcProxy = createClientDatanodeProtocolProxy(addr,
         UserGroupInformation.getCurrentUser(), conf,
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
@@ -143,10 +141,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
       boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Connecting to datanode " + dnAddr + " addr=" + addr);
-    }
-    
+    LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
+
     // Since we're creating a new UserGroupInformation here, we know that no
     // future RPC proxies will be able to re-use the same connection. And
     // usages of this proxy tend to be one-off calls.

+ 13 - 32
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java

@@ -129,18 +129,13 @@ public class DfsClientShmManager implements Closeable {
       ShmId shmId = shm.getShmId();
       Slot slot = shm.allocAndRegisterSlot(blockId);
       if (shm.isFull()) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": pulled the last slot " + slot.getSlotIdx() +
-              " out of " + shm);
-        }
+        LOG.trace("{}: pulled the last slot {} out of {}",
+            this, slot.getSlotIdx(), shm);
         DfsClientShm removedShm = notFull.remove(shmId);
         Preconditions.checkState(removedShm == shm);
         full.put(shmId, shm);
       } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": pulled slot " + slot.getSlotIdx() +
-              " out of " + shm);
-        }
+        LOG.trace("{}: pulled slot {} out of {}", this, slot.getSlotIdx(), shm);
       }
       return slot;
     }
@@ -187,9 +182,7 @@ public class DfsClientShmManager implements Closeable {
           DfsClientShm shm = 
               new DfsClientShm(PBHelperClient.convert(resp.getId()),
                   fis[0], this, peer);
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": createNewShm: created " + shm);
-          }
+          LOG.trace("{}: createNewShm: created {}", this, shm);
           return shm;
         } finally {
           try {
@@ -234,15 +227,11 @@ public class DfsClientShmManager implements Closeable {
         String clientName, ExtendedBlockId blockId) throws IOException {
       while (true) {
         if (closed) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": the DfsClientShmManager has been closed.");
-          }
+          LOG.trace("{}: the DfsClientShmManager has been closed.", this);
           return null;
         }
         if (disabled) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": shared memory segment access is disabled.");
-          }
+          LOG.trace("{}: shared memory segment access is disabled.", this);
           return null;
         }
         // Try to use an existing slot.
@@ -253,9 +242,7 @@ public class DfsClientShmManager implements Closeable {
         // There are no free slots.  If someone is loading more slots, wait
         // for that to finish.
         if (loading) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": waiting for loading to finish...");
-          }
+          LOG.trace("{}: waiting for loading to finish...", this);
           finishedLoading.awaitUninterruptibly();
         } else {
           // Otherwise, load the slot ourselves.
@@ -282,11 +269,9 @@ public class DfsClientShmManager implements Closeable {
             // fired and marked the shm as disconnected.  In this case, we
             // obviously don't want to add the SharedMemorySegment to our list
             // of valid not-full segments.
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(this + ": the UNIX domain socket associated with " +
-                  "this short-circuit memory closed before we could make " +
-                  "use of the shm.");
-            }
+            LOG.debug("{}: the UNIX domain socket associated with this "
+                + "short-circuit memory closed before we could make use of "
+                + "the shm.", this);
           } else {
             notFull.put(shm.getShmId(), shm);
           }
@@ -309,9 +294,7 @@ public class DfsClientShmManager implements Closeable {
         Preconditions.checkState(!full.containsKey(shm.getShmId()));
         Preconditions.checkState(!notFull.containsKey(shm.getShmId()));
         if (shm.isEmpty()) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": freeing empty stale " + shm);
-          }
+          LOG.trace("{}: freeing empty stale {}", this, shm);
           shm.free();
         }
       } else {
@@ -336,10 +319,8 @@ public class DfsClientShmManager implements Closeable {
           // lowest ID, but it could still occur.  In most workloads,
           // fragmentation should not be a major concern, since it doesn't impact
           // peak file descriptor usage or the speed of allocation.
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": shutting down UNIX domain socket for " +
-                "empty " + shm);
-          }
+          LOG.trace("{}: shutting down UNIX domain socket for empty {}",
+              this, shm);
           shutdown(shm);
         } else {
           notFull.put(shmId, shm);

+ 23 - 78
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java

@@ -103,9 +103,7 @@ public class ShortCircuitCache implements Closeable {
         if (ShortCircuitCache.this.closed) return;
         long curMs = Time.monotonicNow();
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(this + ": cache cleaner running at " + curMs);
-        }
+        LOG.debug("{}: cache cleaner running at {}", this, curMs);
 
         int numDemoted = demoteOldEvictableMmaped(curMs);
         int numPurged = 0;
@@ -127,11 +125,9 @@ public class ShortCircuitCache implements Closeable {
           numPurged++;
         }
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug(this + ": finishing cache cleaner run started at " +
-            curMs + ".  Demoted " + numDemoted + " mmapped replicas; " +
-            "purged " + numPurged + " replicas.");
-        }
+        LOG.debug("{}: finishing cache cleaner run started at {}. Demoted {} "
+            + "mmapped replicas; purged {} replicas.",
+            this, curMs, numDemoted, numPurged);
       } finally {
         ShortCircuitCache.this.lock.unlock();
       }
@@ -186,9 +182,7 @@ public class ShortCircuitCache implements Closeable {
 
     @Override
     public void run() {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(ShortCircuitCache.this + ": about to release " + slot);
-      }
+      LOG.trace("{}: about to release {}", ShortCircuitCache.this, slot);
       final DfsClientShm shm = (DfsClientShm)slot.getShm();
       final DomainSocket shmSock = shm.getPeer().getDomainSocket();
       final String path = shmSock.getPath();
@@ -205,9 +199,7 @@ public class ShortCircuitCache implements Closeable {
           String error = resp.hasError() ? resp.getError() : "(unknown)";
           throw new IOException(resp.getStatus().toString() + ": " + error);
         }
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(ShortCircuitCache.this + ": released " + slot);
-        }
+        LOG.trace("{}: released {}", this, slot);
         success = true;
       } catch (IOException e) {
         LOG.error(ShortCircuitCache.this + ": failed to release " +
@@ -433,9 +425,7 @@ public class ShortCircuitCache implements Closeable {
           purgeReason = "purging replica because it is stale.";
         }
         if (purgeReason != null) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(this + ": " + purgeReason);
-          }
+          LOG.debug("{}: {}", this, purgeReason);
           purge(replica);
         }
       }
@@ -677,10 +667,8 @@ public class ShortCircuitCache implements Closeable {
       ShortCircuitReplicaInfo info = null;
       do {
         if (closed) {
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": can't fetchOrCreate " + key +
-                " because the cache is closed.");
-          }
+          LOG.trace("{}: can't fethchOrCreate {} because the cache is closed.",
+              this, key);
           return null;
         }
         Waitable<ShortCircuitReplicaInfo> waitable = replicaInfoMap.get(key);
@@ -688,9 +676,7 @@ public class ShortCircuitCache implements Closeable {
           try {
             info = fetch(key, waitable);
           } catch (RetriableException e) {
-            if (LOG.isDebugEnabled()) {
-              LOG.debug(this + ": retrying " + e.getMessage());
-            }
+            LOG.debug("{}: retrying {}", this, e.getMessage());
             continue;
           }
         }
@@ -721,9 +707,7 @@ public class ShortCircuitCache implements Closeable {
     // ShortCircuitReplica.  So we simply wait for it to complete.
     ShortCircuitReplicaInfo info;
     try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": found waitable for " + key);
-      }
+      LOG.trace("{}: found waitable for {}", this, key);
       info = waitable.await();
     } catch (InterruptedException e) {
       LOG.info(this + ": interrupted while waiting for " + key);
@@ -765,9 +749,7 @@ public class ShortCircuitCache implements Closeable {
     // Handle loading a new replica.
     ShortCircuitReplicaInfo info = null;
     try {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": loading " + key);
-      }
+      LOG.trace("{}: loading {}", this, key);
       info = creator.createShortCircuitReplicaInfo();
     } catch (RuntimeException e) {
       LOG.warn(this + ": failed to load " + key, e);
@@ -777,9 +759,7 @@ public class ShortCircuitCache implements Closeable {
     try {
       if (info.getReplica() != null) {
         // On success, make sure the cache cleaner thread is running.
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + ": successfully loaded " + info.getReplica());
-        }
+        LOG.trace("{}: successfully loaded {}", this, info.getReplica());
         startCacheCleanerThreadIfNeeded();
         // Note: new ShortCircuitReplicas start with a refCount of 2,
         // indicating that both this cache and whoever requested the 
@@ -811,10 +791,8 @@ public class ShortCircuitCache implements Closeable {
           cleanerExecutor.scheduleAtFixedRate(cacheCleaner, rateMs, rateMs,
               TimeUnit.MILLISECONDS);
       cacheCleaner.setFuture(future);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug(this + ": starting cache cleaner thread which will run " +
-          "every " + rateMs + " ms");
-      }
+      LOG.debug("{}: starting cache cleaner thread which will run every {} ms",
+          this, rateMs);
     }
   }
 
@@ -832,17 +810,12 @@ public class ShortCircuitCache implements Closeable {
           long lastAttemptTimeMs = (Long)replica.mmapData;
           long delta = Time.monotonicNow() - lastAttemptTimeMs;
           if (delta < mmapRetryTimeoutMs) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace(this + ": can't create client mmap for " +
-                  replica + " because we failed to " +
-                  "create one just " + delta + "ms ago.");
-            }
+            LOG.trace("{}: can't create client mmap for {} because we failed to"
+                + " create one just {}ms ago.", this, replica, delta);
             return null;
           }
-          if (LOG.isTraceEnabled()) {
-            LOG.trace(this + ": retrying client mmap for " + replica +
-                ", " + delta + " ms after the previous failure.");
-          }
+          LOG.trace("{}: retrying client mmap for {}, {} ms after the previous "
+              + "failure.", this, replica, delta);
         } else if (replica.mmapData instanceof Condition) {
           Condition cond = (Condition)replica.mmapData;
           cond.awaitUninterruptibly();
@@ -965,38 +938,10 @@ public class ShortCircuitCache implements Closeable {
           }
         }
       }
-      if (LOG.isDebugEnabled()) {
-        StringBuilder builder = new StringBuilder();
-        builder.append("visiting ").append(visitor.getClass().getName()).
-            append("with outstandingMmapCount=").append(outstandingMmapCount).
-            append(", replicas=");
-        String prefix = "";
-        for (Entry<ExtendedBlockId, ShortCircuitReplica> entry : replicas.entrySet()) {
-          builder.append(prefix).append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", failedLoads=");
-        for (Entry<ExtendedBlockId, InvalidToken> entry : failedLoads.entrySet()) {
-          builder.append(prefix).append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", evictable=");
-        for (Entry<Long, ShortCircuitReplica> entry : evictable.entrySet()) {
-          builder.append(prefix).append(entry.getKey()).
-              append(":").append(entry.getValue());
-          prefix = ",";
-        }
-        prefix = "";
-        builder.append(", evictableMmapped=");
-        for (Entry<Long, ShortCircuitReplica> entry : evictableMmapped.entrySet()) {
-          builder.append(prefix).append(entry.getKey()).
-              append(":").append(entry.getValue());
-          prefix = ",";
-        }
-        LOG.debug(builder.toString());
-      }
+      LOG.debug("visiting {} with outstandingMmapCount={}, replicas={}, "
+          + "failedLoads={}, evictable={}, evictableMmapped={}",
+          visitor.getClass().getName(), outstandingMmapCount, replicas,
+          failedLoads, evictable, evictableMmapped);
       visitor.visit(outstandingMmapCount, replicas, failedLoads,
             evictable, evictableMmapped);
     } finally {

+ 9 - 24
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java

@@ -154,25 +154,19 @@ public class ShortCircuitReplica {
       // Check staleness by looking at the shared memory area we use to
       // communicate with the DataNode.
       boolean stale = !slot.isValid();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": checked shared memory segment.  isStale=" + stale);
-      }
+      LOG.trace("{}: checked shared memory segment.  isStale={}", this, stale);
       return stale;
     } else {
       // Fall back to old, time-based staleness method.
       long deltaMs = Time.monotonicNow() - creationTimeMs;
       long staleThresholdMs = cache.getStaleThresholdMs();
       if (deltaMs > staleThresholdMs) {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + " is stale because it's " + deltaMs +
-              " ms old, and staleThresholdMs = " + staleThresholdMs);
-        }
+        LOG.trace("{} is stale because it's {} ms old and staleThreadholdMS={}",
+            this, deltaMs, staleThresholdMs);
         return true;
       } else {
-        if (LOG.isTraceEnabled()) {
-          LOG.trace(this + " is not stale because it's only " + deltaMs +
-              " ms old, and staleThresholdMs = " + staleThresholdMs);
-        }
+        LOG.trace("{} is not stale because it's only {} ms old "
+            + "and staleThresholdMs={}",  this, deltaMs, staleThresholdMs);
         return false;
       }
     }
@@ -194,13 +188,8 @@ public class ShortCircuitReplica {
       return false;
     }
     boolean result = slot.addAnchor();
-    if (LOG.isTraceEnabled()) {
-      if (result) {
-        LOG.trace(this + ": added no-checksum anchor to slot " + slot);
-      } else {
-        LOG.trace(this + ": could not add no-checksum anchor to slot " + slot);
-      }
-    }
+    LOG.trace("{}: {} no-checksum anchor to slot {}",
+        this, result ? "added" : "could not add", slot);
     return result;
   }
 
@@ -263,9 +252,7 @@ public class ShortCircuitReplica {
         suffix += "  scheduling " + slot + " for later release.";
       }
     }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("closed " + this + suffix);
-    }
+    LOG.trace("closed {}{}", this, suffix);
   }
 
   public FileInputStream getDataStream() {
@@ -293,9 +280,7 @@ public class ShortCircuitReplica {
       FileChannel channel = dataStream.getChannel();
       MappedByteBuffer mmap = channel.map(MapMode.READ_ONLY, 0, 
           Math.min(Integer.MAX_VALUE, channel.size()));
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(this + ": created mmap of size " + channel.size());
-      }
+      LOG.trace("{}: created mmap of size {}", this, channel.size());
       return mmap;
     } catch (IOException e) {
       LOG.warn(this + ": mmap error", e);

+ 4 - 10
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java

@@ -484,13 +484,9 @@ public class ShortCircuitShm {
         POSIX.MMAP_PROT_READ | POSIX.MMAP_PROT_WRITE, true, mmappedLength);
     this.slots = new Slot[mmappedLength / BYTES_PER_SLOT];
     this.allocatedSlots = new BitSet(slots.length);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("creating " + this.getClass().getSimpleName() +
-          "(shmId=" + shmId +
-          ", mmappedLength=" + mmappedLength +
-          ", baseAddress=" + String.format("%x", baseAddress) +
-          ", slots.length=" + slots.length + ")");
-    }
+    LOG.trace("creating {}(shmId={}, mmappedLength={}, baseAddress={}, "
+        + "slots.length={})", this.getClass().getSimpleName(), shmId,
+        mmappedLength, String.format("%x", baseAddress), slots.length);
   }
 
   public final ShmId getShmId() {
@@ -615,9 +611,7 @@ public class ShortCircuitShm {
         "tried to unregister slot " + slotIdx + ", which was not registered.");
     allocatedSlots.set(slotIdx, false);
     slots[slotIdx] = null;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(this + ": unregisterSlot " + slotIdx);
-    }
+    LOG.trace("{}: unregisterSlot {}", this, slotIdx);
   }
   
   /**

+ 12 - 51
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java

@@ -36,18 +36,6 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 public abstract class ByteArrayManager {
   static final Logger LOG = LoggerFactory.getLogger(ByteArrayManager.class);
-  private static final ThreadLocal<StringBuilder> DEBUG_MESSAGE =
-      new ThreadLocal<StringBuilder>() {
-    protected StringBuilder initialValue() {
-      return new StringBuilder();
-    }
-  };
-
-  private static void logDebugMessage() {
-    final StringBuilder b = DEBUG_MESSAGE.get();
-    LOG.debug(b.toString());
-    b.setLength(0);
-  }
 
   static final int MIN_ARRAY_LENGTH = 32;
   static final byte[] EMPTY_BYTE_ARRAY = {};
@@ -160,27 +148,18 @@ public abstract class ByteArrayManager {
      * via the {@link FixedLengthManager#recycle(byte[])} method.
      */
     synchronized byte[] allocate() throws InterruptedException {
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", ").append(this);
-      }
+      LOG.debug(", {}", this);
       for(; numAllocated >= maxAllocated;) {
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(": wait ...");
-          logDebugMessage();
-        }
+        LOG.debug(": wait ...");
 
         wait();
 
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append("wake up: ").append(this);
-        }
+        LOG.debug("wake up: {}", this);
       }
       numAllocated++;
 
       final byte[] array = freeQueue.poll();
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", recycled? ").append(array != null);
-      }
+      LOG.debug(", recycled? {}", array != null);
       return array != null? array : new byte[byteArrayLength];
     }
 
@@ -194,9 +173,7 @@ public abstract class ByteArrayManager {
     synchronized int recycle(byte[] array) {
       Preconditions.checkNotNull(array);
       Preconditions.checkArgument(array.length == byteArrayLength);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", ").append(this);
-      }
+      LOG.debug(", {}", this);
 
       notify();
       numAllocated--;
@@ -207,9 +184,7 @@ public abstract class ByteArrayManager {
       }
 
       if (freeQueue.size() < maxAllocated - numAllocated) {
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(", freeQueue.offer");
-        }
+        LOG.debug(", freeQueue.offer");
         freeQueue.offer(array);
       }
       return freeQueue.size();
@@ -349,9 +324,7 @@ public abstract class ByteArrayManager {
     public byte[] newByteArray(final int arrayLength)
         throws InterruptedException {
       Preconditions.checkArgument(arrayLength >= 0);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append("allocate(").append(arrayLength).append(")");
-      }
+      LOG.debug("allocate({})", arrayLength);
 
       final byte[] array;
       if (arrayLength == 0) {
@@ -365,18 +338,12 @@ public abstract class ByteArrayManager {
         final FixedLengthManager manager =
             managers.get(powerOfTwo, aboveThreshold);
 
-        if (LOG.isDebugEnabled()) {
-          DEBUG_MESSAGE.get().append(": count=").append(count)
-              .append(aboveThreshold? ", aboveThreshold": ", belowThreshold");
-        }
+        LOG.debug(": count={}, {}Threshold", count,
+            aboveThreshold ? "above" : "below");
         array = manager != null? manager.allocate(): new byte[powerOfTwo];
       }
 
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", return byte[")
-            .append(array.length).append("]");
-        logDebugMessage();
-      }
+      LOG.debug(", return byte[{}]", array.length);
       return array;
     }
 
@@ -391,10 +358,7 @@ public abstract class ByteArrayManager {
     @Override
     public int release(final byte[] array) {
       Preconditions.checkNotNull(array);
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get()
-            .append("recycle: array.length=").append(array.length);
-      }
+      LOG.debug("recycle: array.length={}", array.length);
 
       final int freeQueueSize;
       if (array.length == 0) {
@@ -404,10 +368,7 @@ public abstract class ByteArrayManager {
         freeQueueSize = manager == null? -1: manager.recycle(array);
       }
 
-      if (LOG.isDebugEnabled()) {
-        DEBUG_MESSAGE.get().append(", freeQueueSize=").append(freeQueueSize);
-        logDebugMessage();
-      }
+      LOG.debug(", freeQueueSize={}", freeQueueSize);
       return freeQueueSize;
     }
 

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

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

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java

@@ -182,9 +182,7 @@ public class URLConnectionFactory {
   public URLConnection openConnection(URL url, boolean isSpnego)
       throws IOException, AuthenticationException {
     if (isSpnego) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("open AuthenticatedURL connection {}", url);
-      }
+      LOG.debug("open AuthenticatedURL connection {}", url);
       UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
       final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
       return new AuthenticatedURL(new KerberosUgiAuthenticator(),

+ 7 - 21
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -233,16 +233,12 @@ 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) {
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("Using UGI token: {}", token);
-        }
+        LOG.debug("Using UGI token: {}", token);
         canRefreshDelegationToken = false;
       } else {
         token = getDelegationToken(null);
         if (token != null) {
-          if(LOG.isDebugEnabled()) {
-            LOG.debug("Fetched new token: {}", token);
-          }
+          LOG.debug("Fetched new token: {}", token);
         } else { // security is disabled
           canRefreshDelegationToken = false;
         }
@@ -257,9 +253,7 @@ public class WebHdfsFileSystem extends FileSystem
     boolean replaced = false;
     if (canRefreshDelegationToken) {
       Token<?> token = getDelegationToken(null);
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Replaced expired token: {}", token);
-      }
+      LOG.debug("Replaced expired token: {}", token);
       setDelegationToken(token);
       replaced = (token != null);
     }
@@ -442,9 +436,7 @@ public class WebHdfsFileSystem extends FileSystem
     InetSocketAddress nnAddr = getCurrentNNAddr();
     final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("url={}", url);
-    }
+    LOG.trace("url={}", url);
     return url;
   }
 
@@ -479,9 +471,7 @@ public class WebHdfsFileSystem extends FileSystem
         + Param.toSortedString("&", getAuthParameters(op))
         + Param.toSortedString("&", parameters);
     final URL url = getNamenodeURL(path, query);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("url={}", url);
-    }
+    LOG.trace("url={}", url);
     return url;
   }
 
@@ -769,9 +759,7 @@ public class WebHdfsFileSystem extends FileSystem
       } catch (Exception e) { // catch json parser errors
         final IOException ioe =
             new IOException("Response decoding failure: "+e.toString(), e);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Response decoding failure: {}", e.toString(), e);
-        }
+        LOG.debug("Response decoding failure.", e);
         throw ioe;
       } finally {
         conn.disconnect();
@@ -1242,9 +1230,7 @@ public class WebHdfsFileSystem extends FileSystem
         cancelDelegationToken(delegationToken);
       }
     } catch (IOException ioe) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Token cancel failed: ", ioe);
-      }
+      LOG.debug("Token cancel failed: ", ioe);
     } finally {
       super.close();
     }

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

@@ -1000,6 +1000,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8696. Make the lower and higher watermark in the DN Netty server
     configurable. (Xiaobing Zhou via wheat9)
 
+    HDFS-8971. Remove guards when calling LOG.debug() and LOG.trace() in client
+    package. (Mingliang Liu via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than