You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zh...@apache.org on 2015/09/30 17:41:50 UTC

[50/58] [abbrv] hadoop git commit: HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai.

HDFS-9170. Move libhdfs / fuse-dfs / libwebhdfs to hdfs-client. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d5a9a3da
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d5a9a3da
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d5a9a3da

Branch: refs/heads/HDFS-7285
Commit: d5a9a3daa0224249221ffa7b8bd5751ab2feca56
Parents: 6f335e4
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Sep 29 17:48:29 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Sep 29 17:48:29 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/BlockReaderFactory.java  | 114 ++++++-------------
 .../apache/hadoop/hdfs/BlockReaderLocal.java    |  77 ++++---------
 .../hadoop/hdfs/BlockReaderLocalLegacy.java     |  20 +---
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  36 ++----
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  85 +++++---------
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  36 +++---
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  10 +-
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  47 +++-----
 .../apache/hadoop/hdfs/RemoteBlockReader.java   |   4 +-
 .../apache/hadoop/hdfs/RemoteBlockReader2.java  |  25 ++--
 .../hadoop/hdfs/client/impl/LeaseRenewer.java   |  18 +--
 .../protocol/datatransfer/PacketReceiver.java   |   8 +-
 .../hdfs/protocol/datatransfer/Sender.java      |   6 +-
 .../datatransfer/sasl/DataTransferSaslUtil.java |   8 +-
 .../ClientDatanodeProtocolTranslatorPB.java     |  10 +-
 .../hdfs/shortcircuit/DfsClientShmManager.java  |  45 +++-----
 .../hdfs/shortcircuit/ShortCircuitCache.java    | 101 ++++------------
 .../hdfs/shortcircuit/ShortCircuitReplica.java  |  33 ++----
 .../hdfs/shortcircuit/ShortCircuitShm.java      |  14 +--
 .../hadoop/hdfs/util/ByteArrayManager.java      |  63 ++--------
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |   8 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |   4 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  28 ++---
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 24 files changed, 230 insertions(+), 573 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
index 4f37090..f249692 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ b/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;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
index 62e7af6..10dc35c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
+++ b/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;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
index 5235287..4a1828e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
+++ b/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;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 92d117c..8f3df81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/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() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index 7101753..81e8c27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/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) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 7a40d73..47c4b7e 100755
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index e275afb..71ce7cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/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;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index d1d8d37..4ea1f41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/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();
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
index 15a5bee..017be9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
+++ b/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)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
index 7a7932d..ca31e67 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
+++ b/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();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
index c3d2cfc..8457d65 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
+++ b/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);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
index c4093b1..e6709d9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
+++ b/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;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
index e856211..d2bc348 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
+++ b/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();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
index 256caff..006d304 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
+++ b/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();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
index f764275..24e1dd2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
+++ b/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.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
index f70398a..4ffc108 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/DfsClientShmManager.java
+++ b/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);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
index 52c1a6e..07f5064 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitCache.java
+++ b/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 {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
index 37566e2..38cf22b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitReplica.java
+++ b/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);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
index 78325a3..fa40c15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/shortcircuit/ShortCircuitShm.java
+++ b/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);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
index a9adb7e..e361252 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/ByteArrayManager.java
+++ b/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;
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
index a864d37..870103e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
+++ b/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;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5a9a3da/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
index 4c23241..be5f17d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
+++ b/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(),