Browse Source

HDFS-8979. Clean up checkstyle warnings in hadoop-hdfs-client module. Contributed by Mingliang Liu.

Haohui Mai 9 years ago
parent
commit
1257483ebf
100 changed files with 2103 additions and 2316 deletions
  1. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/CacheFlag.java
  2. 3 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/XAttr.java
  4. 5 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
  5. 2 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java
  6. 21 18
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
  7. 30 38
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  8. 63 54
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderUtil.java
  10. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ClientContext.java
  11. 197 252
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  12. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
  13. 8 17
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
  14. 87 107
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  15. 62 85
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  16. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
  17. 6 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
  18. 53 60
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  19. 141 184
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  20. 2 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java
  21. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
  22. 10 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java
  23. 1 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
  24. 18 21
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PeerCache.java
  25. 73 75
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  26. 62 64
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
  28. 19 20
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
  29. 28 17
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
  30. 5 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
  31. 16 15
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
  32. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
  33. 4 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
  34. 10 10
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
  35. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/package-info.java
  36. 27 22
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
  37. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/DomainPeer.java
  39. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
  40. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
  41. 18 18
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
  42. 13 18
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
  43. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java
  44. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
  45. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
  46. 10 11
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
  47. 4 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
  48. 11 15
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
  49. 10 12
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
  50. 1 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
  51. 6 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
  52. 22 22
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  53. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  54. 4 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java
  55. 4 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  56. 47 30
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  57. 1 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfoWithStorage.java
  58. 3 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeLocalInfo.java
  59. 2 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
  60. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  61. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
  62. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LastBlockWithStatus.java
  63. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  64. 23 26
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  65. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java
  66. 5 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
  67. 28 27
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
  68. 35 36
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  69. 4 12
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
  70. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockConstructionStage.java
  71. 9 9
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
  72. 28 28
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  73. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
  74. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java
  75. 20 22
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
  76. 21 21
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
  77. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
  78. 20 30
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
  79. 61 58
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  80. 4 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
  81. 61 58
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
  82. 52 56
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
  83. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
  84. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java
  85. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
  86. 20 16
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  87. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
  88. 137 184
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  89. 324 320
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  90. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java
  91. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
  92. 17 15
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  93. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/CachingStrategy.java
  94. 11 9
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
  95. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
  96. 4 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java
  97. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java
  98. 14 13
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
  99. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java
  100. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/CacheFlag.java

@@ -34,7 +34,7 @@ public enum CacheFlag {
   FORCE((short) 0x01);
   private final short mode;
 
-  private CacheFlag(short mode) {
+  CacheFlag(short mode) {
     this.mode = mode;
   }
 

+ 3 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java

@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -33,14 +31,13 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 public class HdfsBlockLocation extends BlockLocation {
 
   private final LocatedBlock block;
-  
-  public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) 
-      throws IOException {
+
+  public HdfsBlockLocation(BlockLocation loc, LocatedBlock block) {
     // Initialize with data from passed in BlockLocation
     super(loc);
     this.block = block;
   }
-  
+
   public LocatedBlock getLocatedBlock() {
     return block;
   }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/XAttr.java

@@ -57,12 +57,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
 @InterfaceAudience.Private
 public class XAttr {
 
-  public static enum NameSpace {
+  public enum NameSpace {
     USER,
     TRUSTED,
     SECURITY,
     SYSTEM,
-    RAW;
+    RAW
   }
 
   private final NameSpace ns;

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java

@@ -23,9 +23,9 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
-/** 
-  * This exception is thrown when a read encounters a block that has no locations
-  * associated with it.
+/**
+  * This exception is thrown when a read encounters a block that has no
+  * locations associated with it.
   */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -41,7 +41,8 @@ public class BlockMissingException extends IOException {
    * @param filename name of corrupted file
    * @param description a description of the corruption details
    */
-  public BlockMissingException(String filename, String description, long offset) {
+  public BlockMissingException(String filename, String description,
+      long offset) {
     super(description);
     this.filename = filename;
     this.offset = offset;

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java

@@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
  */
 @InterfaceAudience.Private
 public interface BlockReader extends ByteBufferReadable {
-  
 
   /* same interface as inputStream java.io.InputStream#read()
    * used by DFSInputStream#read()
@@ -55,7 +54,7 @@ public interface BlockReader extends ByteBufferReadable {
    * network I/O.
    * This may return more than what is actually present in the block.
    */
-  int available() throws IOException;
+  int available();
 
   /**
    * Close the block reader.
@@ -84,7 +83,7 @@ public interface BlockReader extends ByteBufferReadable {
    * @return              true only if this is a local read.
    */
   boolean isLocal();
-  
+
   /**
    * @return              true only if this is a short-circuit read.
    *                      All short-circuit reads are also local.

+ 21 - 18
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java

@@ -72,7 +72,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-/** 
+/**
  * Utility class to create BlockReader implementations.
  */
 @InterfaceAudience.Private
@@ -127,7 +127,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   /**
    * The name of this client.
    */
-  private String clientName; 
+  private String clientName;
 
   /**
    * The DataNode we're talking to.
@@ -170,7 +170,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private RemotePeerFactory remotePeerFactory;
 
   /**
-   * UserGroupInformation  to use for legacy block reader local objects, if needed.
+   * UserGroupInformation to use for legacy block reader local objects,
+   * if needed.
    */
   private UserGroupInformation userGroupInformation;
 
@@ -313,7 +314,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * There are a few caches that are important here.
    *
    * The ShortCircuitCache stores file descriptor objects which have been passed
-   * from the DataNode. 
+   * from the DataNode.
    *
    * The DomainSocketFactory stores information about UNIX domain socket paths
    * that we not been able to use in the past, so that we don't waste time
@@ -426,9 +427,9 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       return null;
     }
     if (clientContext.getDisableLegacyBlockReaderLocal()) {
-        PerformanceAdvisory.LOG.debug("{}: can't construct " +
-            "BlockReaderLocalLegacy because " +
-            "disableLegacyBlockReaderLocal is set.", this);
+      PerformanceAdvisory.LOG.debug("{}: can't construct " +
+          "BlockReaderLocalLegacy because " +
+          "disableLegacyBlockReaderLocal is set.", this);
       return null;
     }
     IOException ioe;
@@ -470,7 +471,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       return null;
     }
     ShortCircuitCache cache = clientContext.getShortCircuitCache();
-    ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+    ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(),
+        block.getBlockPoolId());
     ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
     InvalidToken exc = info.getInvalidTokenException();
     if (exc != null) {
@@ -501,14 +503,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    *
    * @return    Null if we could not communicate with the datanode,
    *            a new ShortCircuitReplicaInfo object otherwise.
-   *            ShortCircuitReplicaInfo objects may contain either an InvalidToken
-   *            exception, or a ShortCircuitReplica object ready to use.
+   *            ShortCircuitReplicaInfo objects may contain either an
+   *            InvalidToken exception, or a ShortCircuitReplica object ready to
+   *            use.
    */
   @Override
   public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
     if (createShortCircuitReplicaInfoCallback != null) {
       ShortCircuitReplicaInfo info =
-        createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
+          createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
       if (info != null) return info;
     }
     LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
@@ -548,7 +551,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
           // Handle an I/O error we got when using a newly created socket.
           // We temporarily disable the domain socket path for a few minutes in
           // this case, to prevent wasting more time on it.
-          LOG.warn(this + ": I/O error requesting file descriptors.  " + 
+          LOG.warn(this + ": I/O error requesting file descriptors.  " +
               "Disabling domain socket " + peer.getDomainSocket(), e);
           IOUtilsClient.cleanup(LOG, peer);
           clientContext.getDomainSocketFactory()
@@ -564,11 +567,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * Request file descriptors from a DomainPeer.
    *
    * @param peer   The peer to use for communication.
-   * @param slot   If non-null, the shared memory slot to associate with the 
+   * @param slot   If non-null, the shared memory slot to associate with the
    *               new ShortCircuitReplica.
-   * 
+   *
    * @return  A ShortCircuitReplica object if we could communicate with the
-   *          datanode; null, otherwise. 
+   *          datanode; null, otherwise.
    * @throws  IOException If we encountered an I/O exception while communicating
    *          with the datanode.
    */
@@ -682,7 +685,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         }
         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.
+          // considered less serious because the underlying socket may be stale.
           LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
         } else {
           // Handle an I/O error we got when using a newly created domain peer.
@@ -756,7 +759,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   public static class BlockReaderPeer {
     final Peer peer;
     final boolean fromCache;
-    
+
     BlockReaderPeer(Peer peer, boolean fromCache) {
       this.peer = peer;
       this.fromCache = fromCache;
@@ -800,7 +803,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     }
     try {
       Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
-        datanode);
+          datanode);
       LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
       return new BlockReaderPeer(peer, false);
     } catch (IOException e) {

+ 30 - 38
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java

@@ -52,7 +52,7 @@ import org.slf4j.LoggerFactory;
  * <ul>
  * <li>The client performing short circuit reads must be configured at the
  * datanode.</li>
- * <li>The client gets the file descriptors for the metadata file and the data 
+ * <li>The client gets the file descriptors for the metadata file and the data
  * file for the block using
  * {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
  * </li>
@@ -155,7 +155,7 @@ class BlockReaderLocal implements BlockReader {
    * The Checksum FileChannel.
    */
   private final FileChannel checksumIn;
-  
+
   /**
    * Checksum type and size.
    */
@@ -170,12 +170,12 @@ class BlockReaderLocal implements BlockReader {
    * Name of the block, for logging purposes.
    */
   private final String filename;
-  
+
   /**
    * Block ID and Block Pool ID.
    */
   private final ExtendedBlock block;
-  
+
   /**
    * Cache of Checksum#bytesPerChecksum.
    */
@@ -204,11 +204,11 @@ class BlockReaderLocal implements BlockReader {
    * size of a single chunk, even if {@link #zeroReadaheadRequested} is true.
    * The reason is because we need to do a certain amount of buffering in order
    * to do checksumming.
-   * 
+   *
    * This determines how many bytes we'll use out of dataBuf and checksumBuf.
    * Why do we allocate buffers, and then (potentially) only use part of them?
    * The rationale is that allocating a lot of buffers of different sizes would
-   * make it very difficult for the DirectBufferPool to re-use buffers. 
+   * make it very difficult for the DirectBufferPool to re-use buffers.
    */
   private final int maxReadaheadLength;
 
@@ -335,9 +335,8 @@ class BlockReaderLocal implements BlockReader {
    */
   private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
       throws IOException {
-    TraceScope scope = tracer.newScope(
-        "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")");
-    try {
+    try (TraceScope ignored = tracer.newScope(
+        "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
       int total = 0;
       long startDataPos = dataPos;
       int startBufPos = buf.position();
@@ -358,7 +357,8 @@ class BlockReaderLocal implements BlockReader {
           buf.limit(buf.position());
           buf.position(startBufPos);
           createChecksumBufIfNeeded();
-          int checksumsNeeded = (total + bytesPerChecksum - 1) / bytesPerChecksum;
+          int checksumsNeeded = (total + bytesPerChecksum - 1) /
+              bytesPerChecksum;
           checksumBuf.clear();
           checksumBuf.limit(checksumsNeeded * checksumSize);
           long checksumPos = BlockMetadataHeader.getHeaderSize()
@@ -367,8 +367,8 @@ class BlockReaderLocal implements BlockReader {
             int nRead = checksumIn.read(checksumBuf, checksumPos);
             if (nRead < 0) {
               throw new IOException("Got unexpected checksum file EOF at " +
-                  checksumPos + ", block file position " + startDataPos + " for " +
-                  "block " + block + " of file " + filename);
+                  checksumPos + ", block file position " + startDataPos +
+                  " for block " + block + " of file " + filename);
             }
             checksumPos += nRead;
           }
@@ -380,24 +380,16 @@ class BlockReaderLocal implements BlockReader {
         }
       }
       return total;
-    } finally {
-      scope.close();
     }
   }
 
   private boolean createNoChecksumContext() {
-    if (verifyChecksum) {
-      if (storageType != null && storageType.isTransient()) {
-        // Checksums are not stored for replicas on transient storage.  We do not
-        // anchor, because we do not intend for client activity to block eviction
-        // from transient storage on the DataNode side.
-        return true;
-      } else {
-        return replica.addNoChecksumAnchor();
-      }
-    } else {
-      return true;
-    }
+    return !verifyChecksum ||
+        // Checksums are not stored for replicas on transient storage.  We do
+        // not anchor, because we do not intend for client activity to block
+        // eviction from transient storage on the DataNode side.
+        (storageType != null && storageType.isTransient()) ||
+        replica.addNoChecksumAnchor();
   }
 
   private void releaseNoChecksumContext() {
@@ -453,14 +445,14 @@ class BlockReaderLocal implements BlockReader {
   /**
    * Fill the data buffer.  If necessary, validate the data against the
    * checksums.
-   * 
+   *
    * We always want the offsets of the data contained in dataBuf to be
    * aligned to the chunk boundary.  If we are validating checksums, we
    * accomplish this by seeking backwards in the file until we're on a
    * chunk boundary.  (This is necessary because we can't checksum a
    * partial chunk.)  If we are not validating checksums, we simply only
    * fill the latter part of dataBuf.
-   * 
+   *
    * @param canSkipChecksum  true if we can skip checksumming.
    * @return                 true if we hit EOF.
    * @throws IOException
@@ -473,11 +465,11 @@ class BlockReaderLocal implements BlockReader {
     dataBuf.limit(maxReadaheadLength);
     if (canSkipChecksum) {
       dataBuf.position(slop);
-      fillBuffer(dataBuf, canSkipChecksum);
+      fillBuffer(dataBuf, true);
     } else {
       dataPos -= slop;
       dataBuf.position(0);
-      fillBuffer(dataBuf, canSkipChecksum);
+      fillBuffer(dataBuf, false);
     }
     dataBuf.limit(dataBuf.position());
     dataBuf.position(Math.min(dataBuf.position(), slop));
@@ -501,7 +493,7 @@ class BlockReaderLocal implements BlockReader {
    * efficiency's sake. As described above, all non-checksum-chunk-aligned
    * reads will be served from the slower read path.
    *
-   * @param buf              The buffer to read into. 
+   * @param buf              The buffer to read into.
    * @param canSkipChecksum  True if we can skip checksums.
    */
   private synchronized int readWithBounceBuffer(ByteBuffer buf,
@@ -621,7 +613,7 @@ class BlockReaderLocal implements BlockReader {
   }
 
   @Override
-  public int available() throws IOException {
+  public int available() {
     // We never do network I/O in BlockReaderLocal.
     return Integer.MAX_VALUE;
   }
@@ -660,8 +652,8 @@ class BlockReaderLocal implements BlockReader {
 
   /**
    * Get or create a memory map for this replica.
-   * 
-   * There are two kinds of ClientMmap objects we could fetch here: one that 
+   *
+   * There are two kinds of ClientMmap objects we could fetch here: one that
    * will always read pre-checksummed data, and one that may read data that
    * hasn't been checksummed.
    *
@@ -671,13 +663,13 @@ class BlockReaderLocal implements BlockReader {
    * If we fetch the latter, we don't bother with anchoring.
    *
    * @param opts     The options to use, such as SKIP_CHECKSUMS.
-   * 
+   *
    * @return         null on failure; the ClientMmap otherwise.
    */
   @Override
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     boolean anchor = verifyChecksum &&
-        (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
+        !opts.contains(ReadOption.SKIP_CHECKSUMS);
     if (anchor) {
       if (!createNoChecksumContext()) {
         LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
@@ -696,7 +688,7 @@ class BlockReaderLocal implements BlockReader {
     }
     return clientMmap;
   }
-  
+
   @VisibleForTesting
   boolean getVerifyChecksum() {
     return this.verifyChecksum;
@@ -706,7 +698,7 @@ class BlockReaderLocal implements BlockReader {
   int getMaxReadaheadLength() {
     return this.maxReadaheadLength;
   }
-  
+
   /**
    * Make the replica anchorable.  Normally this can only be done by the
    * DataNode.  This method is only for testing.

+ 63 - 54
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java

@@ -56,10 +56,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * BlockReaderLocalLegacy enables local short circuited reads. If the DFS client is on
- * the same machine as the datanode, then the client can read files directly
- * from the local file system rather than going through the datanode for better
- * performance. <br>
+ * BlockReaderLocalLegacy enables local short circuited reads. If the DFS client
+ * is on the same machine as the datanode, then the client can read files
+ * directly from the local file system rather than going through the datanode
+ * for better performance. <br>
  *
  * This is the legacy implementation based on HDFS-2246, which requires
  * permissions on the datanode to be set so that clients can directly access the
@@ -90,7 +90,8 @@ class BlockReaderLocalLegacy implements BlockReader {
     LocalDatanodeInfo() {
       final int cacheSize = 10000;
       final float hashTableLoadFactor = 0.75f;
-      int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
+      int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor)
+          + 1;
       cache = Collections
           .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
               hashTableCapacity, hashTableLoadFactor, true) {
@@ -123,7 +124,7 @@ class BlockReaderLocalLegacy implements BlockReader {
       }
       return proxy;
     }
-    
+
     private synchronized void resetDatanodeProxy() {
       if (null != proxy) {
         RPC.stopProxy(proxy);
@@ -135,7 +136,8 @@ class BlockReaderLocalLegacy implements BlockReader {
       return cache.get(b);
     }
 
-    private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
+    private void setBlockLocalPathInfo(ExtendedBlock b,
+        BlockLocalPathInfo info) {
       cache.put(b, info);
     }
 
@@ -143,10 +145,11 @@ class BlockReaderLocalLegacy implements BlockReader {
       cache.remove(b);
     }
   }
-  
+
   // Multiple datanodes could be running on the local machine. Store proxies in
   // a map keyed by the ipc port of the datanode.
-  private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap = new HashMap<Integer, LocalDatanodeInfo>();
+  private static final Map<Integer, LocalDatanodeInfo> localDatanodeInfoMap =
+      new HashMap<>();
 
   private final FileInputStream dataIn; // reader for the data file
   private final FileInputStream checksumIn;   // reader for the checksum file
@@ -158,7 +161,7 @@ class BlockReaderLocalLegacy implements BlockReader {
    * checksum read at construction to position the read cursor correctly.
    */
   private int offsetFromChunkBoundary;
-  
+
   private byte[] skipBuf = null;
 
   /**
@@ -188,7 +191,7 @@ class BlockReaderLocalLegacy implements BlockReader {
   static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
       UserGroupInformation userGroupInformation,
       Configuration configuration, String file, ExtendedBlock blk,
-      Token<BlockTokenIdentifier> token, DatanodeInfo node, 
+      Token<BlockTokenIdentifier> token, DatanodeInfo node,
       long startOffset, long length, StorageType storageType,
       Tracer tracer) throws IOException {
     final ShortCircuitConf scConf = conf.getShortCircuitConf();
@@ -234,12 +237,12 @@ class BlockReaderLocalLegacy implements BlockReader {
             new DataInputStream(checksumIn), blk);
         long firstChunkOffset = startOffset
             - (startOffset % checksum.getBytesPerChecksum());
-        localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
-            startOffset, length, pathinfo, checksum, true, dataIn,
-            firstChunkOffset, checksumIn, tracer);
+        localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
+            startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
+            tracer);
       } else {
-        localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk, token,
-            startOffset, length, pathinfo, dataIn, tracer);
+        localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
+            startOffset, dataIn, tracer);
       }
     } catch (IOException e) {
       // remove from cache
@@ -260,7 +263,7 @@ class BlockReaderLocalLegacy implements BlockReader {
     }
     return localBlockReader;
   }
-  
+
   private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
     LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
     if (ldInfo == null) {
@@ -269,19 +272,20 @@ class BlockReaderLocalLegacy implements BlockReader {
     }
     return ldInfo;
   }
-  
+
   private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
       ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
       Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname,
       StorageType storageType) throws IOException {
-    LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node.getIpcPort());
-    BlockLocalPathInfo pathinfo = null;
+    LocalDatanodeInfo localDatanodeInfo =
+        getLocalDatanodeInfo(node.getIpcPort());
+    BlockLocalPathInfo pathinfo;
     ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
         conf, timeout, connectToDnViaHostname);
     try {
       // make RPC to local datanode to find local pathnames of blocks
       pathinfo = proxy.getBlockLocalPathInfo(blk, token);
-      // We cannot cache the path information for a replica on transient storage.
+      // We can't cache the path information for a replica on transient storage.
       // If the replica gets evicted, then it moves to a different path.  Then,
       // our next attempt to read from the cached path would fail to find the
       // file.  Additionally, the failure would cause us to disable legacy
@@ -299,7 +303,7 @@ class BlockReaderLocalLegacy implements BlockReader {
     }
     return pathinfo;
   }
-  
+
   private static int getSlowReadBufferNumChunks(int bufferSizeBytes,
       int bytesPerChecksum) {
     if (bufferSizeBytes < bytesPerChecksum) {
@@ -315,17 +319,15 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
 
   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
-      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
-      long length, BlockLocalPathInfo pathinfo, FileInputStream dataIn,
+      ExtendedBlock block, long startOffset, FileInputStream dataIn,
       Tracer tracer) throws IOException {
-    this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
+    this(conf, hdfsfile, block, startOffset,
         DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
         dataIn, startOffset, null, tracer);
   }
 
   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
-      ExtendedBlock block, Token<BlockTokenIdentifier> token, long startOffset,
-      long length, BlockLocalPathInfo pathinfo, DataChecksum checksum,
+      ExtendedBlock block, long startOffset, DataChecksum checksum,
       boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
       FileInputStream checksumIn, Tracer tracer) throws IOException {
     this.filename = hdfsfile;
@@ -343,17 +345,20 @@ class BlockReaderLocalLegacy implements BlockReader {
 
     final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
         conf.getShortCircuitBufferSize(), bytesPerChecksum);
-    slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
+    slowReadBuff = bufferPool.getBuffer(
+        bytesPerChecksum * chunksPerChecksumRead);
     checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
     // Initially the buffers have nothing to read.
     slowReadBuff.flip();
     checksumBuff.flip();
     boolean success = false;
     try {
-      // Skip both input streams to beginning of the chunk containing startOffset
+      // Skip both input streams to beginning of the chunk containing
+      // startOffset
       IOUtils.skipFully(dataIn, firstChunkOffset);
       if (checksumIn != null) {
-        long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
+        long checkSumOffset = (firstChunkOffset / bytesPerChecksum) *
+            checksumSize;
         IOUtils.skipFully(checksumIn, checkSumOffset);
       }
       success = true;
@@ -371,9 +376,8 @@ class BlockReaderLocalLegacy implements BlockReader {
    */
   private int fillBuffer(FileInputStream stream, ByteBuffer buf)
       throws IOException {
-    TraceScope scope = tracer.
-        newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")");
-    try {
+    try (TraceScope ignored = tracer.
+        newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
       int bytesRead = stream.getChannel().read(buf);
       if (bytesRead < 0) {
         //EOF
@@ -388,11 +392,9 @@ class BlockReaderLocalLegacy implements BlockReader {
         bytesRead += n;
       }
       return bytesRead;
-    } finally {
-      scope.close();
     }
   }
-  
+
   /**
    * Utility method used by read(ByteBuffer) to partially copy a ByteBuffer into
    * another.
@@ -426,7 +428,8 @@ class BlockReaderLocalLegacy implements BlockReader {
       if (slowReadBuff.hasRemaining()) {
         // There are remaining bytes from a small read available. This usually
         // means this read is unaligned, which falls back to the slow path.
-        int fromSlowReadBuff = Math.min(buf.remaining(), slowReadBuff.remaining());
+        int fromSlowReadBuff = Math.min(buf.remaining(),
+            slowReadBuff.remaining());
         writeSlice(slowReadBuff, buf, fromSlowReadBuff);
         nRead += fromSlowReadBuff;
       }
@@ -458,8 +461,10 @@ class BlockReaderLocalLegacy implements BlockReader {
 
       // offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
       // until chunk boundary
-      if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) || offsetFromChunkBoundary > 0) {
-        int toRead = Math.min(buf.remaining(), bytesPerChecksum - offsetFromChunkBoundary);
+      if ((buf.remaining() > 0 && buf.remaining() < bytesPerChecksum) ||
+          offsetFromChunkBoundary > 0) {
+        int toRead = Math.min(buf.remaining(),
+            bytesPerChecksum - offsetFromChunkBoundary);
         int readResult = fillSlowReadBuffer(toRead);
         if (readResult == -1) {
           return nRead;
@@ -470,7 +475,8 @@ class BlockReaderLocalLegacy implements BlockReader {
         }
       }
     } else {
-      // Non-checksummed reads are much easier; we can just fill the buffer directly.
+      // Non-checksummed reads are much easier; we can just fill the buffer
+      // directly.
       nRead = doByteBufferRead(buf);
       if (nRead > 0) {
         buf.position(buf.position() + nRead);
@@ -512,7 +518,7 @@ class BlockReaderLocalLegacy implements BlockReader {
     if (verifyChecksum) {
       assert buf.remaining() % bytesPerChecksum == 0;
     }
-    int dataRead = -1;
+    int dataRead;
 
     int oldpos = buf.position();
     // Read as much as we can into the buffer.
@@ -528,9 +534,10 @@ class BlockReaderLocalLegacy implements BlockReader {
       toChecksum.limit(oldpos + dataRead);
 
       checksumBuff.clear();
-      // Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
+      // Equivalent to
+      // (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
       int numChunks =
-        (toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
+          (toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
       checksumBuff.limit(checksumSize * numChunks);
 
       fillBuffer(checksumIn, checksumBuff);
@@ -571,7 +578,7 @@ class BlockReaderLocalLegacy implements BlockReader {
    * @return the number of bytes available to read, or -1 if EOF.
    */
   private synchronized int fillSlowReadBuffer(int len) throws IOException {
-    int nRead = -1;
+    int nRead;
     if (slowReadBuff.hasRemaining()) {
       // Already got data, good to go.
       nRead = Math.min(len, slowReadBuff.remaining());
@@ -579,7 +586,8 @@ class BlockReaderLocalLegacy implements BlockReader {
       // Round a complete read of len bytes (plus any implicit offset) to the
       // next chunk boundary, since we try and read in multiples of a chunk
       int nextChunk = len + offsetFromChunkBoundary +
-          (bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum));
+          (bytesPerChecksum -
+              ((len + offsetFromChunkBoundary) % bytesPerChecksum));
       int limit = Math.min(nextChunk, slowReadBuff.capacity());
       assert limit % bytesPerChecksum == 0;
 
@@ -598,7 +606,8 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
 
   @Override
-  public synchronized int read(byte[] buf, int off, int len) throws IOException {
+  public synchronized int read(byte[] buf, int off, int len)
+      throws IOException {
     LOG.trace("read off {} len {}", off, len);
     if (!verifyChecksum) {
       return dataIn.read(buf, off, len);
@@ -625,19 +634,19 @@ class BlockReaderLocalLegacy implements BlockReader {
     if (!verifyChecksum) {
       return dataIn.skip(n);
     }
-  
+
     // caller made sure newPosition is not beyond EOF.
     int remaining = slowReadBuff.remaining();
     int position = slowReadBuff.position();
     int newPosition = position + (int)n;
-  
+
     // if the new offset is already read into dataBuff, just reposition
     if (n <= remaining) {
       assert offsetFromChunkBoundary == 0;
       slowReadBuff.position(newPosition);
       return n;
     }
-  
+
     // for small gap, read through to keep the data/checksum in sync
     if (n - remaining <= bytesPerChecksum) {
       slowReadBuff.position(position + remaining);
@@ -647,11 +656,11 @@ class BlockReaderLocalLegacy implements BlockReader {
       int ret = read(skipBuf, 0, (int)(n - remaining));
       return (remaining + ret);
     }
-  
+
     // optimize for big gap: discard the current buffer, skip to
     // the beginning of the appropriate checksum chunk and then
     // read to the middle of that chunk to be in sync with checksums.
-  
+
     // We can't use this.offsetFromChunkBoundary because we need to know how
     // many bytes of the offset were really read. Calling read(..) with a
     // positive this.offsetFromChunkBoundary causes that many bytes to get
@@ -661,7 +670,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
     slowReadBuff.position(slowReadBuff.limit());
     checksumBuff.position(checksumBuff.limit());
-  
+
     IOUtils.skipFully(dataIn, toskip);
     long checkSumOffset = (toskip / bytesPerChecksum) * checksumSize;
     IOUtils.skipFully(checksumIn, checkSumOffset);
@@ -708,7 +717,7 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
 
   @Override
-  public int available() throws IOException {
+  public int available() {
     // We never do network I/O in BlockReaderLocalLegacy.
     return Integer.MAX_VALUE;
   }
@@ -717,7 +726,7 @@ class BlockReaderLocalLegacy implements BlockReader {
   public boolean isLocal() {
     return true;
   }
-  
+
   @Override
   public boolean isShortCircuit() {
     return true;

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

@@ -54,4 +54,4 @@ class BlockReaderUtil {
       off += ret;
     }
   }
-}
+}

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

@@ -35,19 +35,19 @@ import org.slf4j.LoggerFactory;
 
 /**
  * ClientContext contains context information for a client.
- * 
+ *
  * This allows us to share caches such as the socket cache across
  * DFSClient instances.
  */
 @InterfaceAudience.Private
 public class ClientContext {
-  private static final Logger LOG = LoggerFactory.getLogger(ClientContext.class);
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ClientContext.class);
 
   /**
    * Global map of context names to caches contexts.
    */
-  private final static HashMap<String, ClientContext> CACHES =
-      new HashMap<String, ClientContext>();
+  private final static HashMap<String, ClientContext> CACHES = new HashMap<>();
 
   /**
    * Name of context.
@@ -93,7 +93,7 @@ public class ClientContext {
   private volatile boolean disableLegacyBlockReaderLocal = false;
 
   /** Creating byte[] for {@link DFSOutputStream}. */
-  private final ByteArrayManager byteArrayManager;  
+  private final ByteArrayManager byteArrayManager;
 
   /**
    * Whether or not we complained about a DFSClient fetching a CacheContext that
@@ -152,7 +152,7 @@ public class ClientContext {
       if (!printedConfWarning) {
         printedConfWarning = true;
         LOG.warn("Existing client context '" + name + "' does not match " +
-            "requested configuration.  Existing: " + existing + 
+            "requested configuration.  Existing: " + existing +
             ", Requested: " + requested);
       }
     }

File diff suppressed because it is too large
+ 197 - 252
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java


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

@@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
  * Used for injecting faults in DFSClient and DFSOutputStream tests.
- * Calls into this are a no-op in production code. 
+ * Calls into this are a no-op in production code.
  */
 @VisibleForTesting
 @InterfaceAudience.Private

+ 8 - 17
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java

@@ -69,8 +69,8 @@ public class DFSInotifyEventInputStream {
     this(namenode, tracer, namenode.getCurrentEditLogTxid());
   }
 
-  DFSInotifyEventInputStream(ClientProtocol namenode,
-        Tracer tracer, long lastReadTxid) throws IOException {
+  DFSInotifyEventInputStream(ClientProtocol namenode, Tracer tracer,
+      long lastReadTxid) {
     this.namenode = namenode;
     this.it = Iterators.emptyIterator();
     this.lastReadTxid = lastReadTxid;
@@ -94,8 +94,7 @@ public class DFSInotifyEventInputStream {
    * The next available batch of events will be returned.
    */
   public EventBatch poll() throws IOException, MissingEventsException {
-    TraceScope scope = tracer.newScope("inotifyPoll");
-    try {
+    try (TraceScope ignored = tracer.newScope("inotifyPoll")) {
       // need to keep retrying until the NN sends us the latest committed txid
       if (lastReadTxid == -1) {
         LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
@@ -119,7 +118,7 @@ public class DFSInotifyEventInputStream {
           }
         } else {
           LOG.debug("poll(): read no edits from the NN when requesting edits " +
-            "after txid {}", lastReadTxid);
+              "after txid {}", lastReadTxid);
           return null;
         }
       }
@@ -130,8 +129,6 @@ public class DFSInotifyEventInputStream {
       } else {
         return null;
       }
-    } finally {
-      scope.close();
     }
   }
 
@@ -175,9 +172,8 @@ public class DFSInotifyEventInputStream {
    */
   public EventBatch poll(long time, TimeUnit tu) throws IOException,
       InterruptedException, MissingEventsException {
-    TraceScope scope = tracer.newScope("inotifyPollWithTimeout");
-    EventBatch next = null;
-    try {
+    EventBatch next;
+    try (TraceScope ignored = tracer.newScope("inotifyPollWithTimeout")) {
       long initialTime = Time.monotonicNow();
       long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
       long nextWait = INITIAL_WAIT_MS;
@@ -195,8 +191,6 @@ public class DFSInotifyEventInputStream {
             nextWait);
         Thread.sleep(nextWait);
       }
-    } finally {
-      scope.close();
     }
     return next;
   }
@@ -212,9 +206,8 @@ public class DFSInotifyEventInputStream {
    */
   public EventBatch take() throws IOException, InterruptedException,
       MissingEventsException {
-    TraceScope scope = tracer.newScope("inotifyTake");
-    EventBatch next = null;
-    try {
+    EventBatch next;
+    try (TraceScope ignored = tracer.newScope("inotifyTake")) {
       int nextWaitMin = INITIAL_WAIT_MS;
       while ((next = poll()) == null) {
         // sleep for a random period between nextWaitMin and nextWaitMin * 2
@@ -225,8 +218,6 @@ public class DFSInotifyEventInputStream {
         // the maximum sleep is 2 minutes
         nextWaitMin = Math.min(60000, nextWaitMin * 2);
       }
-    } finally {
-      scope.close();
     }
 
     return next;

+ 87 - 107
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -55,11 +55,9 @@ import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -85,13 +83,15 @@ import org.apache.htrace.core.Tracer;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import javax.annotation.Nonnull;
+
 /****************************************************************
- * DFSInputStream provides bytes from a named file.  It handles 
+ * DFSInputStream provides bytes from a named file.  It handles
  * negotiation of the namenode and various datanodes as necessary.
  ****************************************************************/
 @InterfaceAudience.Private
 public class DFSInputStream extends FSInputStream
-implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
+    implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     HasEnhancedByteBufferAccess, CanUnbuffer {
   @VisibleForTesting
   public static boolean tcpReadsDisabledForTesting = false;
@@ -128,7 +128,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
   /**
    * Track the ByteBuffers that we have handed out to readers.
-   * 
+   *
    * The value type can be either ByteBufferPool or ClientMmap, depending on
    * whether we this is a memory-mapped buffer or not.
    */
@@ -137,7 +137,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   private synchronized IdentityHashStore<ByteBuffer, Object>
         getExtendedReadBuffers() {
     if (extendedReadBuffers == null) {
-      extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
+      extendedReadBuffers = new IdentityHashStore<>(0);
     }
     return extendedReadBuffers;
   }
@@ -177,7 +177,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     public long getTotalShortCircuitBytesRead() {
       return totalShortCircuitBytesRead;
     }
-    
+
     /**
      * @return The total number of zero-copy bytes read.
      */
@@ -191,7 +191,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     public long getRemoteBytesRead() {
       return totalBytesRead - totalLocalBytesRead;
     }
-    
+
     void addRemoteBytes(long amt) {
       this.totalBytesRead += amt;
     }
@@ -220,7 +220,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       this.totalShortCircuitBytesRead = 0;
       this.totalZeroCopyBytesRead = 0;
     }
-    
+
     private long totalBytesRead;
 
     private long totalLocalBytesRead;
@@ -229,7 +229,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
     private long totalZeroCopyBytesRead;
   }
-  
+
   /**
    * This variable tracks the number of failures since the start of the
    * most recent user-facing operation. That is to say, it should be reset
@@ -243,19 +243,19 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    */
   protected int failures = 0;
 
-  /* XXX Use of CocurrentHashMap is temp fix. Need to fix 
+  /* XXX Use of CocurrentHashMap is temp fix. Need to fix
    * parallel accesses to DFSInputStream (through ptreads) properly */
   private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
-             new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
+             new ConcurrentHashMap<>();
 
   private byte[] oneByteBuf; // used for 'int read()'
 
   void addToDeadNodes(DatanodeInfo dnInfo) {
     deadNodes.put(dnInfo, dnInfo);
   }
-  
+
   DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
+      LocatedBlocks locatedBlocks) throws IOException {
     this.dfsClient = dfsClient;
     this.verifyChecksum = verifyChecksum;
     this.src = src;
@@ -270,8 +270,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Grab the open-file info from namenode
    * @param refreshLocatedBlocks whether to re-fetch locatedblocks
    */
-  void openInfo(boolean refreshLocatedBlocks) throws IOException,
-      UnresolvedLinkException {
+  void openInfo(boolean refreshLocatedBlocks) throws IOException {
     final DfsClientConf conf = dfsClient.getConf();
     synchronized(infoLock) {
       lastBlockBeingWrittenLength =
@@ -344,7 +343,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
         final long len = readBlockLength(last);
         last.getBlock().setNumBytes(len);
-        lastBlockBeingWrittenLength = len; 
+        lastBlockBeingWrittenLength = len;
       }
     }
 
@@ -357,30 +356,30 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   private long readBlockLength(LocatedBlock locatedblock) throws IOException {
     assert locatedblock != null : "LocatedBlock cannot be null";
     int replicaNotFoundCount = locatedblock.getLocations().length;
-    
+
     final DfsClientConf conf = dfsClient.getConf();
     for(DatanodeInfo datanode : locatedblock.getLocations()) {
       ClientDatanodeProtocol cdp = null;
-      
+
       try {
         cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
             dfsClient.getConfiguration(), conf.getSocketTimeout(),
             conf.isConnectToDnViaHostname(), locatedblock);
-        
+
         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
-        
+
         if (n >= 0) {
           return n;
         }
       }
       catch(IOException ioe) {
         if (ioe instanceof RemoteException &&
-          (((RemoteException) ioe).unwrapRemoteException() instanceof
-            ReplicaNotFoundException)) {
+            (((RemoteException) ioe).unwrapRemoteException() instanceof
+                ReplicaNotFoundException)) {
           // special case : replica might not be on the DN, treat as 0 length
           replicaNotFoundCount--;
         }
-        
+
         DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
               + " for block {}", datanode, locatedblock.getBlock(), ioe);
       } finally {
@@ -400,7 +399,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
     throw new IOException("Cannot obtain block length for " + locatedblock);
   }
-  
+
   public long getFileLength() {
     synchronized(infoLock) {
       return locatedBlocks == null? 0:
@@ -424,7 +423,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /**
-   * Returns the block containing the target position. 
+   * Returns the block containing the target position.
    */
   synchronized public ExtendedBlock getCurrentBlock() {
     if (currentLocatedBlock == null){
@@ -443,7 +442,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /**
    * Get block at the specified position.
    * Fetch it from the namenode if not cached.
-   * 
+   *
    * @param offset block corresponding to this offset in file is returned
    * @return located block
    * @throws IOException
@@ -526,12 +525,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         blocks = getFinalizedBlockRange(offset,
           Math.min(length, lengthOfCompleteBlk - offset));
       } else {
-        blocks = new ArrayList<LocatedBlock>(1);
+        blocks = new ArrayList<>(1);
       }
 
       // get the blocks from incomplete block range
       if (readLengthPastCompleteBlk) {
-         blocks.add(locatedBlocks.getLastLocatedBlock());
+        blocks.add(locatedBlocks.getLastLocatedBlock());
       }
 
       return blocks;
@@ -547,7 +546,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       long offset, long length) throws IOException {
     synchronized(infoLock) {
       assert (locatedBlocks != null) : "locatedBlocks is null";
-      List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
+      List<LocatedBlock> blockRange = new ArrayList<>();
       // search cached blocks first
       int blockIdx = locatedBlocks.findBlock(offset);
       if (blockIdx < 0) { // block is not cached
@@ -591,10 +590,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     //
     // Connect to best DataNode for desired Block, with potential offset
     //
-    DatanodeInfo chosenNode = null;
+    DatanodeInfo chosenNode;
     int refetchToken = 1; // only need to get a new access token once
     int refetchEncryptionKey = 1; // only need to get a new encryption key once
-    
+
     boolean connectFailedOnce = false;
 
     while (true) {
@@ -639,7 +638,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         } else {
           connectFailedOnce = true;
           DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
-            + ", add to deadNodes and continue. " + ex, ex);
+              + ", add to deadNodes and continue. " + ex, ex);
           // Put chosen node into dead list, continue
           addToDeadNodes(chosenNode);
         }
@@ -722,8 +721,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * strategy-agnostic.
    */
   interface ReaderStrategy {
-    public int doRead(BlockReader blockReader, int off, int len)
-        throws ChecksumException, IOException;
+    int doRead(BlockReader blockReader, int off, int len)
+        throws IOException;
 
     /**
      * Copy data from the src ByteBuffer into the read buffer.
@@ -733,7 +732,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
      * @param length Useful only when the ReadStrategy is based on a byte array.
      *               Indicate the length of the data to copy.
      */
-    public int copyFrom(ByteBuffer src, int offset, int length);
+    int copyFrom(ByteBuffer src, int offset, int length);
   }
 
   protected void updateReadStatistics(ReadStatistics readStatistics,
@@ -749,7 +748,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       }
     }
   }
-  
+
   /**
    * Used to read bytes into a byte[]
    */
@@ -762,7 +761,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
     @Override
     public int doRead(BlockReader blockReader, int off, int len)
-          throws ChecksumException, IOException {
+        throws IOException {
       int nRead = blockReader.read(buf, off, len);
       updateReadStatistics(readStatistics, nRead, blockReader);
       return nRead;
@@ -787,7 +786,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
     @Override
     public int doRead(BlockReader blockReader, int off, int len)
-        throws ChecksumException, IOException {
+        throws IOException {
       int oldpos = buf.position();
       int oldlimit = buf.limit();
       boolean success = false;
@@ -805,7 +804,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           buf.position(oldpos);
           buf.limit(oldlimit);
         }
-      } 
+      }
     }
 
     @Override
@@ -821,12 +820,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /* This is a used by regular read() and handles ChecksumExceptions.
    * name readBuffer() is chosen to imply similarity to readBuffer() in
    * ChecksumFileSystem
-   */ 
+   */
   private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
     IOException ioe;
-    
+
     /* we retry current node only once. So this is set to true only here.
      * Intention is to handle one common case of an error that is not a
      * failure on datanode or client : when DataNode closes the connection
@@ -842,7 +841,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       } catch ( ChecksumException ce ) {
         DFSClient.LOG.warn("Found Checksum error for "
             + getCurrentBlock() + " from " + currentNode
-            + " at " + ce.getPos());        
+            + " at " + ce.getPos());
         ioe = ce;
         retryCurrentNode = false;
         // we want to remember which block replicas we have tried
@@ -856,12 +855,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
         ioe = e;
       }
-      boolean sourceFound = false;
+      boolean sourceFound;
       if (retryCurrentNode) {
         /* possibly retry the same node so that transient errors don't
          * result in application level failures (e.g. Datanode could have
          * closed the connection because the client is idle for too long).
-         */ 
+         */
         sourceFound = seekToBlockSource(pos);
       } else {
         addToDeadNodes(currentNode);
@@ -879,8 +878,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if (closed.get()) {
       throw new IOException("Stream closed");
     }
-    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
-      = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
     failures = 0;
     if (pos < getFileLength()) {
       int retries = 2;
@@ -899,7 +897,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             }
           }
           int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
-          
+
           if (result >= 0) {
             pos += result;
           } else {
@@ -911,7 +909,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           }
           return result;
         } catch (ChecksumException ce) {
-          throw ce;            
+          throw ce;
         } catch (IOException e) {
           if (retries == 1) {
             DFSClient.LOG.warn("DFS Read", e);
@@ -924,7 +922,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         } finally {
           // Check if need to report block replicas corruption either read
           // was successful or ChecksumException occured.
-          reportCheckSumFailure(corruptedBlockMap, 
+          reportCheckSumFailure(corruptedBlockMap,
               currentLocatedBlock.getLocations().length);
         }
       }
@@ -936,26 +934,21 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Read the entire buffer.
    */
   @Override
-  public synchronized int read(final byte buf[], int off, int len) throws IOException {
+  public synchronized int read(@Nonnull final byte buf[], int off, int len)
+      throws IOException {
     ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
-    TraceScope scope =
-        dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src);
-    try {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("DFSInputStream#byteArrayRead", src)) {
       return readWithStrategy(byteArrayReader, off, len);
-    } finally {
-      scope.close();
     }
   }
 
   @Override
   public synchronized int read(final ByteBuffer buf) throws IOException {
     ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
-    TraceScope scope =
-        dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src);
-    try {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("DFSInputStream#byteBufferRead", src)){
       return readWithStrategy(byteBufferReader, 0, buf.remaining());
-    } finally {
-      scope.close();
     }
   }
 
@@ -965,11 +958,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    */
   protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
-    Set<DatanodeInfo> dnSet = null;
+    Set<DatanodeInfo> dnSet;
     if((corruptedBlockMap.containsKey(blk))) {
       dnSet = corruptedBlockMap.get(blk);
     }else {
-      dnSet = new HashSet<DatanodeInfo>();
+      dnSet = new HashSet<>();
     }
     if (!dnSet.contains(node)) {
       dnSet.add(node);
@@ -985,7 +978,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         return result;
       } else {
         String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
-          deadNodes, ignoredNodes);
+            deadNodes, ignoredNodes);
         String blockInfo = block.getBlock() + " file=" + src;
         if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
           String description = "Could not obtain block: " + blockInfo;
@@ -1011,7 +1004,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // Also at the second retry, the waiting window is expanded to 6000 ms
           // alleviating the request rate from the server. Similarly the 3rd retry
           // will wait 6000ms grace period before retry and the waiting window is
-          // expanded to 9000ms. 
+          // expanded to 9000ms.
           final int timeWindow = dfsClient.getConf().getTimeWindow();
           double waitTime = timeWindow * failures +       // grace period for the last round of attempt
               // expanding time window for each failure
@@ -1019,7 +1012,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
               ThreadLocalRandom.current().nextDouble();
           DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
           Thread.sleep((long)waitTime);
-        } catch (InterruptedException iex) {
+        } catch (InterruptedException ignored) {
         }
         deadNodes.clear(); //2nd option is to remove only nodes[blockId]
         openInfo(true);
@@ -1122,14 +1115,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       public ByteBuffer call() throws Exception {
         byte[] buf = bb.array();
         int offset = bb.position();
-        TraceScope scope = dfsClient.getTracer().
-            newScope("hedgedRead" + hedgedReadId, parentSpanId);
-        try {
+        try (TraceScope ignored = dfsClient.getTracer().
+            newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
           actualGetFromOneDataNode(datanode, block, start, end, buf,
               offset, corruptedBlockMap);
           return bb;
-        } finally {
-          scope.close();
         }
       }
     };
@@ -1271,12 +1261,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
     final DfsClientConf conf = dfsClient.getConf();
-    ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
+    ArrayList<Future<ByteBuffer>> futures = new ArrayList<>();
     CompletionService<ByteBuffer> hedgedService =
-        new ExecutorCompletionService<ByteBuffer>(
-        dfsClient.getHedgedReadsThreadPool());
-    ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
-    ByteBuffer bb = null;
+        new ExecutorCompletionService<>(dfsClient.getHedgedReadsThreadPool());
+    ArrayList<DatanodeInfo> ignored = new ArrayList<>();
+    ByteBuffer bb;
     int len = (int) (end - start + 1);
     int hedgedReadId = 0;
     block = refreshLocatedBlock(block);
@@ -1308,11 +1297,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // Ignore this node on next go around.
           ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
-          continue; // no need to refresh block locations
-        } catch (InterruptedException e) {
+          // continue; no need to refresh block locations
+        } catch (InterruptedException | ExecutionException e) {
           // Ignore
-        } catch (ExecutionException e) {
-          // Ignore already logged in the call.
         }
       } else {
         // We are starting up a 'hedged' read. We have a read already
@@ -1377,10 +1364,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       ByteBuffer bb = future.get();
       futures.remove(future);
       return bb;
-    } catch (ExecutionException e) {
-      // already logged in the Callable
-      futures.remove(future);
-    } catch (CancellationException ce) {
+    } catch (ExecutionException | CancellationException e) {
       // already logged in the Callable
       futures.remove(future);
     }
@@ -1401,7 +1385,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
   /**
    * Should the block access token be refetched on an exception
-   * 
+   *
    * @param ex Exception received
    * @param targetAddr Target datanode address from where exception was received
    * @return true if block access token has expired or invalid and it should be
@@ -1429,23 +1413,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
   /**
    * Read bytes starting from the specified position.
-   * 
+   *
    * @param position start read from this position
    * @param buffer read buffer
    * @param offset offset into buffer
    * @param length number of bytes to read
-   * 
+   *
    * @return actual number of bytes read
    */
   @Override
   public int read(long position, byte[] buffer, int offset, int length)
       throws IOException {
-    TraceScope scope = dfsClient.
-        newPathTraceScope("DFSInputStream#byteArrayPread", src);
-    try {
+    try (TraceScope ignored = dfsClient.
+        newPathTraceScope("DFSInputStream#byteArrayPread", src)) {
       return pread(position, buffer, offset, length);
-    } finally {
-      scope.close();
     }
   }
 
@@ -1465,13 +1446,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if ((position + length) > filelen) {
       realLen = (int)(filelen - position);
     }
-    
+
     // determine the block and byte range within the block
     // corresponding to position and realLen
     List<LocatedBlock> blockRange = getBlockRange(position, realLen);
     int remaining = realLen;
-    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
-      = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
     for (LocatedBlock blk : blockRange) {
       long targetStart = position - blk.getStartOffset();
       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
@@ -1500,12 +1480,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     return realLen;
   }
-  
+
   /**
    * DFSInputStream reports checksum failure.
    * Case I : client has tried multiple data nodes and at least one of the
    * attempts has succeeded. We report the other failures as corrupted block to
-   * namenode. 
+   * namenode.
    * Case II: client has tried out all data nodes, but all failed. We
    * only report if the total number of replica is 1. We do not
    * report otherwise since this maybe due to the client is a handicapped client
@@ -1514,7 +1494,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @param dataNodeCount number of data nodes who contains the block replicas
    */
   protected void reportCheckSumFailure(
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap, 
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
       int dataNodeCount) {
     if (corruptedBlockMap.isEmpty()) {
       return;
@@ -1581,8 +1561,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           } else {
             // The range was already checked. If the block reader returns
             // something unexpected instead of throwing an exception, it is
-            // most likely a bug. 
-            String errMsg = "BlockReader failed to seek to " + 
+            // most likely a bug.
+            String errMsg = "BlockReader failed to seek to " +
                 targetPos + ". Instead, it seeked to " + pos + ".";
             DFSClient.LOG.warn(errMsg);
             throw new IOException(errMsg);
@@ -1608,10 +1588,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     currentNode = blockSeekTo(targetPos);
     return true;
   }
-  
+
   /**
    * Seek to given position on a node other than the current node.  If
-   * a node other than the current node is found, then returns true. 
+   * a node other than the current node is found, then returns true.
    * If another node could not be found, then returns false.
    */
   @Override
@@ -1624,7 +1604,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     DatanodeInfo oldNode = currentNode;
     DatanodeInfo newNode = blockSeekTo(targetPos);
     if (!markedDead) {
-      /* remove it from deadNodes. blockSeekTo could have cleared 
+      /* remove it from deadNodes. blockSeekTo could have cleared
        * deadNodes and added currentNode again. Thats ok. */
       deadNodes.remove(oldNode);
     }
@@ -1635,7 +1615,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       return false;
     }
   }
-      
+
   /**
    */
   @Override
@@ -1712,7 +1692,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
   protected void closeCurrentBlockReaders() {
     if (blockReader == null) return;
-    // Close the current block reader so that the new caching settings can 
+    // Close the current block reader so that the new caching settings can
     // take effect immediately.
     try {
       blockReader.close();
@@ -1748,11 +1728,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * zero-copy read.
    */
   private static final ByteBuffer EMPTY_BUFFER =
-    ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
+      ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
 
   @Override
   public synchronized ByteBuffer read(ByteBufferPool bufferPool,
-      int maxLength, EnumSet<ReadOption> opts) 
+      int maxLength, EnumSet<ReadOption> opts)
           throws IOException, UnsupportedOperationException {
     if (maxLength == 0) {
       return EMPTY_BUFFER;

+ 62 - 85
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -90,7 +89,7 @@ public class DFSOutputStream extends FSOutputSummer
     implements Syncable, CanSetDropBehind {
   static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
   /**
-   * Number of times to retry creating a file when there are transient 
+   * Number of times to retry creating a file when there are transient
    * errors (typically related to encryption zones and KeyProvider operations).
    */
   @VisibleForTesting
@@ -121,8 +120,9 @@ public class DFSOutputStream extends FSOutputSummer
   private FileEncryptionInfo fileEncryptionInfo;
 
   /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
-  protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
-      long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
+  protected DFSPacket createPacket(int packetSize, int chunksPerPkt,
+      long offsetInBlock, long seqno, boolean lastPacketInBlock)
+      throws InterruptedIOException {
     final byte[] buf;
     final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
 
@@ -159,9 +159,7 @@ public class DFSOutputStream extends FSOutputSummer
       return null;
     }
     DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
-    for (int i = 0; i < currentNodes.length; i++) {
-      value[i] = currentNodes[i];
-    }
+    System.arraycopy(currentNodes, 0, value, 0, currentNodes.length);
     return value;
   }
 
@@ -178,9 +176,9 @@ public class DFSOutputStream extends FSOutputSummer
     }
     return checksum;
   }
- 
-  private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
-      HdfsFileStatus stat, DataChecksum checksum) throws IOException {
+
+  private DFSOutputStream(DFSClient dfsClient, String src,
+      Progressable progress, HdfsFileStatus stat, DataChecksum checksum) {
     super(getChecksum4Compute(checksum, stat));
     this.dfsClient = dfsClient;
     this.src = src;
@@ -188,7 +186,7 @@ public class DFSOutputStream extends FSOutputSummer
     this.blockSize = stat.getBlockSize();
     this.blockReplication = stat.getReplication();
     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
-    this.cachingStrategy = new AtomicReference<CachingStrategy>(
+    this.cachingStrategy = new AtomicReference<>(
         dfsClient.getDefaultWriteCachingStrategy());
     if (progress != null) {
       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
@@ -202,8 +200,9 @@ public class DFSOutputStream extends FSOutputSummer
     }
     if (blockSize % bytesPerChecksum != 0) {
       throw new HadoopIllegalArgumentException("Invalid values: "
-          + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
-          + ") must divide block size (=" + blockSize + ").");
+          + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY
+          + " (=" + bytesPerChecksum + ") must divide block size (=" +
+          blockSize + ").");
     }
     this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
   }
@@ -215,7 +214,8 @@ public class DFSOutputStream extends FSOutputSummer
     this(dfsClient, src, progress, stat, checksum);
     this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
 
-    computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
+    computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
+        bytesPerChecksum);
 
     streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
         cachingStrategy, byteArrayManager, favoredNodes);
@@ -223,11 +223,10 @@ public class DFSOutputStream extends FSOutputSummer
 
   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
       FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
-      short replication, long blockSize, Progressable progress, int buffersize,
+      short replication, long blockSize, Progressable progress,
       DataChecksum checksum, String[] favoredNodes) throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("newStreamForCreate", src);
-    try {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("newStreamForCreate", src)) {
       HdfsFileStatus stat = null;
 
       // Retry the create if we get a RetryStartFileException up to a maximum
@@ -238,7 +237,7 @@ public class DFSOutputStream extends FSOutputSummer
         shouldRetry = false;
         try {
           stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
-              new EnumSetWritable<CreateFlag>(flag), createParent, replication,
+              new EnumSetWritable<>(flag), createParent, replication,
               blockSize, SUPPORTED_CRYPTO_VERSIONS);
           break;
         } catch (RemoteException re) {
@@ -273,8 +272,6 @@ public class DFSOutputStream extends FSOutputSummer
           flag, progress, checksum, favoredNodes);
       out.start();
       return out;
-    } finally {
-      scope.close();
     }
   }
 
@@ -294,17 +291,17 @@ public class DFSOutputStream extends FSOutputSummer
     // The last partial block of the file has to be filled.
     if (!toNewBlock && lastBlock != null) {
       // indicate that we are appending to an existing block
-      streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
-          cachingStrategy, byteArrayManager);
+      streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress,
+          checksum, cachingStrategy, byteArrayManager);
       getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
       adjustPacketChunkSize(stat);
       getStreamer().setPipelineInConstruction(lastBlock);
     } else {
       computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
           bytesPerChecksum);
-      streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
-          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
-          favoredNodes);
+      streamer = new DataStreamer(stat,
+          lastBlock != null ? lastBlock.getBlock() : null, dfsClient, src,
+          progress, checksum, cachingStrategy, byteArrayManager, favoredNodes);
     }
   }
 
@@ -345,18 +342,15 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
-      EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
-      LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
-      String[] favoredNodes) throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("newStreamForAppend", src);
-    try {
+      EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
+      HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
+      throws IOException {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("newStreamForAppend", src)) {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
           progress, lastBlock, stat, checksum, favoredNodes);
       out.start();
       return out;
-    } finally {
-      scope.close();
     }
   }
 
@@ -479,36 +473,28 @@ public class DFSOutputStream extends FSOutputSummer
    * of the DNs but not necessarily in the DN's OS buffers.
    *
    * It is a synchronous operation. When it returns,
-   * it guarantees that flushed data become visible to new readers. 
-   * It is not guaranteed that data has been flushed to 
-   * persistent store on the datanode. 
+   * it guarantees that flushed data become visible to new readers.
+   * It is not guaranteed that data has been flushed to
+   * persistent store on the datanode.
    * Block allocations are persisted on namenode.
    */
   @Override
   public void hflush() throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("hflush", src);
-    try {
+    try (TraceScope ignored = dfsClient.newPathTraceScope("hflush", src)) {
       flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
-    } finally {
-      scope.close();
     }
   }
 
   @Override
   public void hsync() throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("hsync", src);
-    try {
+    try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
       flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
-    } finally {
-      scope.close();
     }
   }
   
   /**
-   * The expected semantics is all data have flushed out to all replicas 
-   * and all replicas have done posix fsync equivalent - ie the OS has 
+   * The expected semantics is all data have flushed out to all replicas
+   * and all replicas have done posix fsync equivalent - ie the OS has
    * flushed it to the disk device (but the disk may have it in its cache).
    * 
    * Note that only the current block is flushed to the disk device.
@@ -520,12 +506,8 @@ public class DFSOutputStream extends FSOutputSummer
    *          whether or not to update the block length in NameNode.
    */
   public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("hsync", src);
-    try {
+    try (TraceScope ignored = dfsClient.newPathTraceScope("hsync", src)) {
       flushOrSync(true, syncFlags);
-    } finally {
-      scope.close();
     }
   }
 
@@ -630,13 +612,14 @@ public class DFSOutputStream extends FSOutputSummer
           dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
               lastBlockLength);
         } catch (IOException ioe) {
-          DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
-          // If we got an error here, it might be because some other thread called
-          // close before our hflush completed. In that case, we should throw an
-          // exception that the stream is closed.
+          DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src,
+              ioe);
+          // If we got an error here, it might be because some other thread
+          // called close before our hflush completed. In that case, we should
+          // throw an exception that the stream is closed.
           checkClosed();
-          // If we aren't closed but failed to sync, we should expose that to the
-          // caller.
+          // If we aren't closed but failed to sync, we should expose that to
+          // the caller.
           throw ioe;
         }
       }
@@ -647,9 +630,9 @@ public class DFSOutputStream extends FSOutputSummer
         }
       }
     } catch (InterruptedIOException interrupt) {
-      // This kind of error doesn't mean that the stream itself is broken - just the
-      // flushing thread got interrupted. So, we shouldn't close down the writer,
-      // but instead just propagate the error
+      // This kind of error doesn't mean that the stream itself is broken - just
+      // the flushing thread got interrupted. So, we shouldn't close down the
+      // writer, but instead just propagate the error
       throw interrupt;
     } catch (IOException e) {
       DFSClient.LOG.warn("Error while syncing", e);
@@ -691,8 +674,8 @@ public class DFSOutputStream extends FSOutputSummer
   }
   
   /**
-   * Waits till all existing data is flushed and confirmations 
-   * received from datanodes. 
+   * Waits till all existing data is flushed and confirmations
+   * received from datanodes.
    */
   protected void flushInternal() throws IOException {
     long toWaitFor;
@@ -715,7 +698,7 @@ public class DFSOutputStream extends FSOutputSummer
   }
   
   /**
-   * Aborts this output stream and releases any system 
+   * Aborts this output stream and releases any system
    * resources associated with this stream.
    */
   synchronized void abort() throws IOException {
@@ -723,7 +706,7 @@ public class DFSOutputStream extends FSOutputSummer
       return;
     }
     getStreamer().getLastException().set(new IOException("Lease timeout of "
-        + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
+        + (dfsClient.getConf().getHdfsTimeout() / 1000) + " seconds expired."));
     closeThreads(true);
     dfsClient.endFileLease(fileId);
   }
@@ -753,17 +736,14 @@ public class DFSOutputStream extends FSOutputSummer
   }
   
   /**
-   * Closes this output stream and releases any system 
+   * Closes this output stream and releases any system
    * resources associated with this stream.
    */
   @Override
   public synchronized void close() throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("DFSOutputStream#close", src);
-    try {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("DFSOutputStream#close", src)) {
       closeImpl();
-    } finally {
-      scope.close();
     }
   }
 
@@ -788,20 +768,18 @@ public class DFSOutputStream extends FSOutputSummer
       // get last block before destroying the streamer
       ExtendedBlock lastBlock = getStreamer().getBlock();
       closeThreads(false);
-      TraceScope scope = dfsClient.getTracer().newScope("completeFile");
-      try {
+      try (TraceScope ignored =
+               dfsClient.getTracer().newScope("completeFile")) {
         completeFile(lastBlock);
-      } finally {
-        scope.close();
       }
       dfsClient.endFileLease(fileId);
-    } catch (ClosedChannelException e) {
+    } catch (ClosedChannelException ignored) {
     } finally {
       setClosed();
     }
   }
 
-  // should be called holding (this) lock since setTestFilename() may 
+  // should be called holding (this) lock since setTestFilename() may
   // be called during unit tests
   protected void completeFile(ExtendedBlock last) throws IOException {
     long localstart = Time.monotonicNow();
@@ -817,12 +795,11 @@ public class DFSOutputStream extends FSOutputSummer
         if (!dfsClient.clientRunning
             || (hdfsTimeout > 0
                 && localstart + hdfsTimeout < Time.monotonicNow())) {
-            String msg = "Unable to close file because dfsclient " +
-                          " was unable to contact the HDFS servers." +
-                          " clientRunning " + dfsClient.clientRunning +
-                          " hdfsTimeout " + hdfsTimeout;
-            DFSClient.LOG.info(msg);
-            throw new IOException(msg);
+          String msg = "Unable to close file because dfsclient " +
+              " was unable to contact the HDFS servers. clientRunning " +
+              dfsClient.clientRunning + " hdfsTimeout " + hdfsTimeout;
+          DFSClient.LOG.info(msg);
+          throw new IOException(msg);
         }
         try {
           if (retries == 0) {

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

@@ -139,7 +139,6 @@ class DFSPacket {
   /**
    * Write the full packet, including the header, to the given output stream.
    *
-   * @param stm
    * @throws IOException
    */
   synchronized void writeTo(DataOutputStream stm) throws IOException {
@@ -173,15 +172,18 @@ class DFSPacket {
 
     // corrupt the data for testing.
     if (DFSClientFaultInjector.get().corruptPacket()) {
-      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^=
+          0xff;
     }
 
     // Write the now contiguous full packet to the output stream.
-    stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
+    stm.write(buf, headerStart,
+        header.getSerializedSize() + checksumLen + dataLen);
 
     // undo corruption.
     if (DFSClientFaultInjector.get().uncorruptPacket()) {
-      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^=
+          0xff;
     }
   }
 
@@ -193,8 +195,6 @@ class DFSPacket {
 
   /**
    * Release the buffer in this packet to ByteArrayManager.
-   *
-   * @param bam
    */
   synchronized void releaseBuffer(ByteArrayManager bam) {
     bam.release(buf);

+ 6 - 7
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java

@@ -303,8 +303,8 @@ public class DFSUtilClient {
    * @param keys Set of keys to look for in the order of preference
    * @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
    */
-  static Map<String, Map<String, InetSocketAddress>>
-    getAddresses(Configuration conf, String defaultAddress, String... keys) {
+  static Map<String, Map<String, InetSocketAddress>> getAddresses(
+      Configuration conf, String defaultAddress, String... keys) {
     Collection<String> nameserviceIds = getNameServiceIds(conf);
     return getAddressesForNsIds(conf, nameserviceIds, defaultAddress, keys);
   }
@@ -317,8 +317,7 @@ public class DFSUtilClient {
    *
    * @return a map(nameserviceId to map(namenodeId to InetSocketAddress))
    */
-  static Map<String, Map<String, InetSocketAddress>>
-    getAddressesForNsIds(
+  static Map<String, Map<String, InetSocketAddress>> getAddressesForNsIds(
       Configuration conf, Collection<String> nsIds, String defaultAddress,
       String... keys) {
     // Look for configurations of the form <key>[.<nameserviceId>][.<namenodeId>]
@@ -326,7 +325,7 @@ public class DFSUtilClient {
     Map<String, Map<String, InetSocketAddress>> ret = Maps.newLinkedHashMap();
     for (String nsId : emptyAsSingletonNull(nsIds)) {
       Map<String, InetSocketAddress> isas =
-        getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
+          getAddressesForNameserviceId(conf, nsId, defaultAddress, keys);
       if (!isas.isEmpty()) {
         ret.put(nsId, isas);
       }
@@ -533,7 +532,7 @@ public class DFSUtilClient {
 
   public static Peer peerFromSocket(Socket socket)
       throws IOException {
-    Peer peer = null;
+    Peer peer;
     boolean success = false;
     try {
       // TCP_NODELAY is crucial here because of bad interactions between
@@ -560,7 +559,7 @@ public class DFSUtilClient {
       return peer;
     } finally {
       if (!success) {
-        if (peer != null) peer.close();
+        // peer is always null so no need to call peer.close().
         socket.close();
       }
     }

+ 53 - 60
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java

@@ -41,7 +41,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@@ -80,7 +79,6 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.Sampler;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanId;
 import org.apache.htrace.core.TraceScope;
@@ -95,6 +93,8 @@ import com.google.common.cache.RemovalNotification;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
+
 /*********************************************************************
  *
  * The DataStreamer class is responsible for sending data packets to the
@@ -136,7 +136,8 @@ class DataStreamer extends Daemon {
     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());
+    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(),
+        conf.getSocketTimeout());
     sock.setSoTimeout(timeout);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
     LOG.debug("Send buf size {}", sock.getSendBufferSize());
@@ -427,7 +428,6 @@ class DataStreamer extends Daemon {
    * Construct a data streamer for appending to the last partial block
    * @param lastBlock last block of the file to be appended
    * @param stat status of the file to be appended
-   * @throws IOException if error occurs
    */
   DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
                String src, Progressable progress, DataChecksum checksum,
@@ -517,7 +517,7 @@ class DataStreamer extends Daemon {
         // process datanode IO errors if any
         boolean doSleep = processDatanodeError();
 
-        final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
+        final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2;
         synchronized (dataQueue) {
           // wait for a packet to be sent.
           long now = Time.monotonicNow();
@@ -620,9 +620,8 @@ class DataStreamer extends Daemon {
         LOG.debug("DataStreamer block {} sending packet {}", block, one);
 
         // write out data to remote datanode
-        TraceScope writeScope = dfsClient.getTracer().
-            newScope("DataStreamer#writeTo", spanId);
-        try {
+        try (TraceScope ignored = dfsClient.getTracer().
+            newScope("DataStreamer#writeTo", spanId)) {
           one.writeTo(blockStream);
           blockStream.flush();
         } catch (IOException e) {
@@ -634,8 +633,6 @@ class DataStreamer extends Daemon {
           // will be taken out then.
           errorState.markFirstNodeIfNotMarked();
           throw e;
-        } finally {
-          writeScope.close();
         }
         lastPacket = Time.monotonicNow();
 
@@ -725,9 +722,8 @@ class DataStreamer extends Daemon {
    * @throws IOException
    */
   void waitForAckedSeqno(long seqno) throws IOException {
-    TraceScope scope = dfsClient.getTracer().
-        newScope("waitForAckedSeqno");
-    try {
+    try (TraceScope ignored = dfsClient.getTracer().
+        newScope("waitForAckedSeqno")) {
       LOG.debug("Waiting for ack for: {}", seqno);
       long begin = Time.monotonicNow();
       try {
@@ -747,15 +743,13 @@ class DataStreamer extends Daemon {
           }
         }
         checkClosed();
-      } catch (ClosedChannelException e) {
+      } catch (ClosedChannelException cce) {
       }
       long duration = Time.monotonicNow() - begin;
       if (duration > dfsclientSlowLogThresholdMs) {
         LOG.warn("Slow waitForAckedSeqno took " + duration
             + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
       }
-    } finally {
-      scope.close();
     }
   }
 
@@ -802,7 +796,7 @@ class DataStreamer extends Daemon {
         }
         checkClosed();
         queuePacket(packet);
-      } catch (ClosedChannelException e) {
+      } catch (ClosedChannelException ignored) {
       }
     }
   }
@@ -901,10 +895,7 @@ class DataStreamer extends Daemon {
       assert false;
     }
 
-    if (addr != null && NetUtils.isLocalAddress(addr)) {
-      return true;
-    }
-    return false;
+    return addr != null && NetUtils.isLocalAddress(addr);
   }
 
   //
@@ -1137,17 +1128,15 @@ class DataStreamer extends Daemon {
   ) throws IOException {
     if (nodes.length != original.length + 1) {
       throw new IOException(
-          new StringBuilder()
-              .append("Failed to replace a bad datanode on the existing pipeline ")
-              .append("due to no more good datanodes being available to try. ")
-              .append("(Nodes: current=").append(Arrays.asList(nodes))
-              .append(", original=").append(Arrays.asList(original)).append("). ")
-              .append("The current failed datanode replacement policy is ")
-              .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
-              .append("a client may configure this via '")
-              .append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
-              .append("' in its configuration.")
-              .toString());
+          "Failed to replace a bad datanode on the existing pipeline "
+              + "due to no more good datanodes being available to try. "
+              + "(Nodes: current=" + Arrays.asList(nodes)
+              + ", original=" + Arrays.asList(original) + "). "
+              + "The current failed datanode replacement policy is "
+              + dfsClient.dtpReplaceDatanodeOnFailure
+              + ", and a client may configure this via '"
+              + BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
+              + "' in its configuration.");
     }
     for(int i = 0; i < nodes.length; i++) {
       int j = 0;
@@ -1196,7 +1185,7 @@ class DataStreamer extends Daemon {
     final StorageType[] originalTypes = storageTypes;
     final String[] originalIDs = storageIDs;
     IOException caughtException = null;
-    ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
+    ArrayList<DatanodeInfo> exclude = new ArrayList<>(failed);
     while (tried < 3) {
       LocatedBlock lb;
       //get a new datanode
@@ -1235,7 +1224,8 @@ class DataStreamer extends Daemon {
 
   private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
                         final StorageType[] targetStorageTypes,
-                        final Token<BlockTokenIdentifier> blockToken) throws IOException {
+                        final Token<BlockTokenIdentifier> blockToken)
+      throws IOException {
     //transfer replica to the new datanode
     Socket sock = null;
     DataOutputStream out = null;
@@ -1246,7 +1236,8 @@ class DataStreamer extends Daemon {
 
       // transfer timeout multiplier based on the transfer size
       // One per 200 packets = 12.8MB. Minimum is 2.
-      int multi = 2 + (int)(bytesSent/dfsClient.getConf().getWritePacketSize())/200;
+      int multi = 2 + (int)(bytesSent /dfsClient.getConf().getWritePacketSize())
+          / 200;
       final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
 
       OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
@@ -1413,13 +1404,14 @@ class DataStreamer extends Daemon {
         // good reports should follow bad ones, if client committed
         // with those nodes.
         Thread.sleep(2000);
-      } catch (InterruptedException ie) {}
+      } catch (InterruptedException ignored) {
+      }
     }
   }
 
-  LocatedBlock updateBlockForPipeline() throws IOException {
-    return dfsClient.namenode.updateBlockForPipeline(
-        block, dfsClient.clientName);
+  private LocatedBlock updateBlockForPipeline() throws IOException {
+    return dfsClient.namenode.updateBlockForPipeline(block,
+        dfsClient.clientName);
   }
 
   /** update pipeline at the namenode */
@@ -1437,12 +1429,12 @@ class DataStreamer extends Daemon {
    * Must get block ID and the IDs of the destinations from the namenode.
    * Returns the list of target datanodes.
    */
-  private LocatedBlock nextBlockOutputStream() throws IOException {
-    LocatedBlock lb = null;
-    DatanodeInfo[] nodes = null;
-    StorageType[] storageTypes = null;
+  protected LocatedBlock nextBlockOutputStream() throws IOException {
+    LocatedBlock lb;
+    DatanodeInfo[] nodes;
+    StorageType[] storageTypes;
     int count = dfsClient.getConf().getNumBlockWriteRetry();
-    boolean success = false;
+    boolean success;
     ExtendedBlock oldBlock = block;
     do {
       errorState.reset();
@@ -1493,7 +1485,6 @@ class DataStreamer extends Daemon {
       LOG.info("nodes are empty for write pipeline of " + block);
       return false;
     }
-    Status pipelineStatus = SUCCESS;
     String firstBadLink = "";
     boolean checkRestart = false;
     if (LOG.isDebugEnabled()) {
@@ -1528,25 +1519,26 @@ class DataStreamer extends Daemon {
         // Xmit header info to datanode
         //
 
-        BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
+        BlockConstructionStage bcs = recoveryFlag ?
+            stage.getRecoveryStage() : stage;
 
         // We cannot change the block length in 'block' as it counts the number
         // of bytes ack'ed.
         ExtendedBlock blockCopy = new ExtendedBlock(block);
         blockCopy.setNumBytes(stat.getBlockSize());
 
-        boolean[] targetPinnings = getPinnings(nodes, true);
+        boolean[] targetPinnings = getPinnings(nodes);
         // send the request
         new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
             dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
             nodes.length, block.getNumBytes(), bytesSent, newGS,
             checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
-            (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
+            (targetPinnings != null && targetPinnings[0]), targetPinnings);
 
         // receive ack for connect
         BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
             PBHelperClient.vintPrefixed(blockReplyStream));
-        pipelineStatus = resp.getStatus();
+        Status pipelineStatus = resp.getStatus();
         firstBadLink = resp.getFirstBadLink();
 
         // Got an restart OOB ack.
@@ -1559,7 +1551,7 @@ class DataStreamer extends Daemon {
           checkRestart = true;
           throw new IOException("A datanode is restarting.");
         }
-		
+
         String logInfo = "ack with firstBadLink as " + firstBadLink;
         DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
 
@@ -1571,7 +1563,8 @@ class DataStreamer extends Daemon {
         if (!errorState.isRestartingNode()) {
           LOG.info("Exception in createBlockOutputStream", ie);
         }
-        if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+        if (ie instanceof InvalidEncryptionKeyException &&
+            refetchEncryptionKey > 0) {
           LOG.info("Will fetch a new encryption key and retry, "
               + "encryption key was invalid when connecting to "
               + nodes[0] + " : " + ie);
@@ -1593,14 +1586,15 @@ class DataStreamer extends Daemon {
             }
           }
         } else {
-          assert checkRestart == false;
+          assert !checkRestart;
           errorState.setBadNodeIndex(0);
         }
 
         final int i = errorState.getBadNodeIndex();
         // Check whether there is a restart worth waiting for.
         if (checkRestart && shouldWaitForRestart(i)) {
-          errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
+          errorState.initRestartingNode(i, "Datanode " + i + " is restarting: "
+              + nodes[i]);
         }
         errorState.setError(true);
         lastException.set(ie);
@@ -1610,7 +1604,6 @@ class DataStreamer extends Daemon {
           IOUtils.closeSocket(s);
           s = null;
           IOUtils.closeStream(out);
-          out = null;
           IOUtils.closeStream(blockReplyStream);
           blockReplyStream = null;
         }
@@ -1619,19 +1612,18 @@ class DataStreamer extends Daemon {
     }
   }
 
-  private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
+  private boolean[] getPinnings(DatanodeInfo[] nodes) {
     if (favoredNodes == null) {
       return null;
     } else {
       boolean[] pinnings = new boolean[nodes.length];
-      HashSet<String> favoredSet =
-          new HashSet<String>(Arrays.asList(favoredNodes));
+      HashSet<String> favoredSet = new HashSet<>(Arrays.asList(favoredNodes));
       for (int i = 0; i < nodes.length; i++) {
         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
         LOG.debug("{} was chosen by name node (favored={}).",
             nodes[i].getXferAddrWithHostname(), pinnings[i]);
       }
-      if (shouldLog && !favoredSet.isEmpty()) {
+      if (!favoredSet.isEmpty()) {
         // There is one or more favored nodes that were not allocated.
         LOG.warn("These favored nodes were specified but not chosen: "
             + favoredSet + " Specified favored nodes: "
@@ -1777,7 +1769,7 @@ class DataStreamer extends Daemon {
    * For heartbeat packets, create buffer directly by new byte[]
    * since heartbeats should not be blocked.
    */
-  private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
+  private DFSPacket createHeartbeatPacket() {
     final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
     return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
   }
@@ -1789,7 +1781,8 @@ class DataStreamer extends Daemon {
         .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
           @Override
           public void onRemoval(
-              RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
+              @Nonnull RemovalNotification<DatanodeInfo, DatanodeInfo>
+                  notification) {
             LOG.info("Removing node " + notification.getKey()
                 + " from the excluded nodes list");
           }

File diff suppressed because it is too large
+ 141 - 184
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java


+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java

@@ -40,7 +40,7 @@ final public class ExtendedBlockId {
   public static ExtendedBlockId fromExtendedBlock(ExtendedBlock block) {
     return new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
   }
-  
+
   public ExtendedBlockId(long blockId, String bpId) {
     this.blockId = blockId;
     this.bpId = bpId;
@@ -76,7 +76,6 @@ final public class ExtendedBlockId {
 
   @Override
   public String toString() {
-    return new StringBuilder().append(blockId).
-        append("_").append(bpId).toString();
+    return String.valueOf(blockId) + "_" + bpId;
   }
 }

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

@@ -79,7 +79,7 @@ public final class ExternalBlockReader implements BlockReader {
   }
 
   @Override
-  public int available() throws IOException {
+  public int available() {
     // We return the amount of bytes between the current offset and the visible
     // length.  Some of the other block readers return a shorter length than
     // that.  The only advantage to returning a shorter length is that the

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs;
 
-import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.concurrent.Callable;
@@ -37,10 +36,13 @@ import com.google.common.cache.RemovalNotification;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nonnull;
+
 @InterfaceAudience.Private
 public class KeyProviderCache {
 
-  public static final Logger LOG = LoggerFactory.getLogger(KeyProviderCache.class);
+  public static final Logger LOG = LoggerFactory.getLogger(
+      KeyProviderCache.class);
 
   private final Cache<URI, KeyProvider> cache;
 
@@ -50,14 +52,14 @@ public class KeyProviderCache {
         .removalListener(new RemovalListener<URI, KeyProvider>() {
           @Override
           public void onRemoval(
-              RemovalNotification<URI, KeyProvider> notification) {
+              @Nonnull RemovalNotification<URI, KeyProvider> notification) {
             try {
+              assert notification.getValue() != null;
               notification.getValue().close();
             } catch (Throwable e) {
               LOG.error(
                   "Error closing KeyProvider with uri ["
                       + notification.getKey() + "]", e);
-              ;
             }
           }
         })
@@ -83,8 +85,8 @@ public class KeyProviderCache {
   }
 
   private URI createKeyProviderURI(Configuration conf) {
-    final String providerUriStr =
-        conf.getTrimmed(HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
+    final String providerUriStr = conf.getTrimmed(
+        HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
     // No provider set in conf
     if (providerUriStr.isEmpty()) {
       LOG.error("Could not find uri with key ["
@@ -104,9 +106,9 @@ public class KeyProviderCache {
   }
 
   @VisibleForTesting
-  public void setKeyProvider(Configuration conf, KeyProvider keyProvider)
-      throws IOException {
+  public void setKeyProvider(Configuration conf, KeyProvider keyProvider) {
     URI uri = createKeyProviderURI(conf);
+    assert uri != null;
     cache.put(uri, keyProvider);
   }
 }

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

@@ -297,13 +297,11 @@ public class NameNodeProxiesClient {
    * @param failoverProxyProvider Failover proxy provider
    * @return an object containing both the proxy and the associated
    *         delegation token service it corresponds to
-   * @throws IOException
    */
   @SuppressWarnings("unchecked")
   public static <T> ProxyAndInfo<T> createHAProxy(
       Configuration conf, URI nameNodeUri, Class<T> xface,
-      AbstractNNFailoverProxyProvider<T> failoverProxyProvider)
-      throws IOException {
+      AbstractNNFailoverProxyProvider<T> failoverProxyProvider) {
     Preconditions.checkNotNull(failoverProxyProvider);
     // HA case
     DfsClientConf config = new DfsClientConf(conf);

+ 18 - 21
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PeerCache.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.util.IOUtilsClient;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
@@ -46,16 +45,16 @@ import org.slf4j.LoggerFactory;
 @VisibleForTesting
 public class PeerCache {
   private static final Logger LOG = LoggerFactory.getLogger(PeerCache.class);
-  
+
   private static class Key {
     final DatanodeID dnID;
     final boolean isDomain;
-    
+
     Key(DatanodeID dnID, boolean isDomain) {
       this.dnID = dnID;
       this.isDomain = isDomain;
     }
-    
+
     @Override
     public boolean equals(Object o) {
       if (!(o instanceof Key)) {
@@ -70,7 +69,7 @@ public class PeerCache {
       return dnID.hashCode() ^ (isDomain ? 1 : 0);
     }
   }
-  
+
   private static class Value {
     private final Peer peer;
     private final long time;
@@ -92,10 +91,10 @@ public class PeerCache {
   private Daemon daemon;
   /** A map for per user per datanode. */
   private final LinkedListMultimap<Key, Value> multimap =
-    LinkedListMultimap.create();
+      LinkedListMultimap.create();
   private final int capacity;
   private final long expiryPeriod;
-  
+
   public PeerCache(int c, long e) {
     this.capacity = c;
     this.expiryPeriod = e;
@@ -107,17 +106,17 @@ public class PeerCache {
          expiryPeriod + " when cache is enabled.");
     }
   }
- 
+
   private boolean isDaemonStarted() {
-    return (daemon == null)? false: true;
+    return daemon != null;
   }
 
   private synchronized void startExpiryDaemon() {
     // start daemon only if not already started
-    if (isDaemonStarted() == true) {
+    if (isDaemonStarted()) {
       return;
     }
-    
+
     daemon = new Daemon(new Runnable() {
       @Override
       public void run() {
@@ -144,7 +143,7 @@ public class PeerCache {
    * @param isDomain     Whether to retrieve a DomainPeer or not.
    *
    * @return             An open Peer connected to the DN, or null if none
-   *                     was found. 
+   *                     was found.
    */
   public Peer get(DatanodeID dnId, boolean isDomain) {
 
@@ -215,12 +214,11 @@ public class PeerCache {
   private synchronized void evictExpired(long expiryPeriod) {
     while (multimap.size() != 0) {
       Iterator<Entry<Key, Value>> iter =
-        multimap.entries().iterator();
+          multimap.entries().iterator();
       Entry<Key, Value> entry = iter.next();
       // if oldest socket expired, remove it
-      if (entry == null || 
-        Time.monotonicNow() - entry.getValue().getTime() <
-        expiryPeriod) {
+      if (entry == null ||
+          Time.monotonicNow() - entry.getValue().getTime() < expiryPeriod) {
         break;
       }
       IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
@@ -235,8 +233,7 @@ public class PeerCache {
     // We can get the oldest element immediately, because of an interesting
     // property of LinkedListMultimap: its iterator traverses entries in the
     // order that they were added.
-    Iterator<Entry<Key, Value>> iter =
-      multimap.entries().iterator();
+    Iterator<Entry<Key, Value>> iter = multimap.entries().iterator();
     if (!iter.hasNext()) {
       throw new IllegalStateException("Cannot evict from empty cache! " +
         "capacity: " + capacity);
@@ -247,8 +244,8 @@ public class PeerCache {
   }
 
   /**
-   * Periodically check in the cache and expire the entries
-   * older than expiryPeriod minutes
+   * Periodically check in the cache and expire the entries older than
+   * expiryPeriod minutes.
    */
   private void run() throws InterruptedException {
     for(long lastExpiryTime = Time.monotonicNow();
@@ -274,7 +271,7 @@ public class PeerCache {
     }
     multimap.clear();
   }
-  
+
   @VisibleForTesting
   void close() {
     clear();

+ 73 - 75
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java

@@ -55,7 +55,8 @@ import org.slf4j.LoggerFactory;
 
 /**
  * @deprecated this is an old implementation that is being left around
- * in case any issues spring up with the new {@link RemoteBlockReader2} implementation.
+ * in case any issues spring up with the new {@link RemoteBlockReader2}
+ * implementation.
  * It will be removed in the next release.
  */
 @InterfaceAudience.Private
@@ -79,7 +80,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   private final long blockId;
 
   /** offset in block of of first chunk - may be less than startOffset
-      if startOffset is not chunk-aligned */
+   if startOffset is not chunk-aligned */
   private final long firstChunkOffset;
 
   private final int bytesPerChecksum;
@@ -91,7 +92,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
    * at the beginning so that the read can begin on a chunk boundary.
    */
   private final long bytesNeededToFinish;
-  
+
   /**
    * True if we are reading from a local DataNode.
    */
@@ -99,17 +100,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
   private boolean eos = false;
   private boolean sentStatusCode = false;
-  
+
   ByteBuffer checksumBytes = null;
   /** Amount of unread data in the current received packet */
   int dataLeft = 0;
-  
+
   private final PeerCache peerCache;
 
   private final Tracer tracer;
-  
+
   /* FSInputChecker interface */
-  
+
   /* same interface as inputStream java.io.InputStream#read()
    * used by DFSInputStream#read()
    * This violates one rule when there is a checksum error:
@@ -118,9 +119,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
    * the checksum.
    */
   @Override
-  public synchronized int read(byte[] buf, int off, int len) 
-                               throws IOException {
-    
+  public synchronized int read(byte[] buf, int off, int len)
+      throws IOException {
+
     // This has to be set here, *before* the skip, since we can
     // hit EOS during the skip, in the case that our entire read
     // is smaller than the checksum chunk.
@@ -135,7 +136,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
         throw new IOException("Could not skip required number of bytes");
       }
     }
-    
+
     int nRead = super.read(buf, off, len);
 
     // if eos was set in the previous read, send a status code to the DN
@@ -152,7 +153,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   @Override
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
-     * in majority of the cases?. This one throws. */  
+     * in majority of the cases?. This one throws. */
     long nSkipped = 0;
     while (nSkipped < n) {
       int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
@@ -168,18 +169,18 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   @Override
   public int read() throws IOException {
     throw new IOException("read() is not expected to be invoked. " +
-                          "Use read(buf, off, len) instead.");
+        "Use read(buf, off, len) instead.");
   }
-  
+
   @Override
   public boolean seekToNewSource(long targetPos) throws IOException {
-    /* Checksum errors are handled outside the BlockReader. 
-     * DFSInputStream does not always call 'seekToNewSource'. In the 
+    /* Checksum errors are handled outside the BlockReader.
+     * DFSInputStream does not always call 'seekToNewSource'. In the
      * case of pread(), it just tries a different replica without seeking.
-     */ 
+     */
     return false;
   }
-  
+
   @Override
   public void seek(long pos) throws IOException {
     throw new IOException("Seek() is not supported in BlockInputChecker");
@@ -188,17 +189,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   @Override
   protected long getChunkPosition(long pos) {
     throw new RuntimeException("getChunkPosition() is not supported, " +
-                               "since seek is not required");
+        "since seek is not required");
   }
-  
+
   /**
-   * Makes sure that checksumBytes has enough capacity 
-   * and limit is set to the number of checksum bytes needed 
+   * Makes sure that checksumBytes has enough capacity
+   * and limit is set to the number of checksum bytes needed
    * to be read.
    */
   private void adjustChecksumBytes(int dataLen) {
-    int requiredSize = 
-      ((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
+    int requiredSize =
+        ((dataLen + bytesPerChecksum - 1)/bytesPerChecksum)*checksumSize;
     if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
       checksumBytes =  ByteBuffer.wrap(new byte[requiredSize]);
     } else {
@@ -206,42 +207,39 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     }
     checksumBytes.limit(requiredSize);
   }
-  
+
   @Override
-  protected synchronized int readChunk(long pos, byte[] buf, int offset, 
-                                       int len, byte[] checksumBuf) 
-                                       throws IOException {
-    TraceScope scope = tracer.
-        newScope("RemoteBlockReader#readChunk(" + blockId + ")");
-    try {
+  protected synchronized int readChunk(long pos, byte[] buf, int offset,
+      int len, byte[] checksumBuf)
+      throws IOException {
+    try (TraceScope ignored = tracer.newScope(
+        "RemoteBlockReader#readChunk(" + blockId + ")")) {
       return readChunkImpl(pos, buf, offset, len, checksumBuf);
-    } finally {
-      scope.close();
     }
   }
 
   private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
-                                     int len, byte[] checksumBuf)
-                                     throws IOException {
+      int len, byte[] checksumBuf)
+      throws IOException {
     // Read one chunk.
     if (eos) {
       // Already hit EOF
       return -1;
     }
-    
+
     // Read one DATA_CHUNK.
     long chunkOffset = lastChunkOffset;
     if ( lastChunkLen > 0 ) {
       chunkOffset += lastChunkLen;
     }
-    
+
     // pos is relative to the start of the first chunk of the read.
     // chunkOffset is relative to the start of the block.
     // This makes sure that the read passed from FSInputChecker is the
     // for the same chunk we expect to be reading from the DN.
     if ( (pos + firstChunkOffset) != chunkOffset ) {
-      throw new IOException("Mismatch in pos : " + pos + " + " + 
-                            firstChunkOffset + " != " + chunkOffset);
+      throw new IOException("Mismatch in pos : " + pos + " + " +
+          firstChunkOffset + " != " + chunkOffset);
     }
 
     // Read next packet if the previous packet has been read completely.
@@ -254,8 +252,8 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
       // Sanity check the lengths
       if (!header.sanityCheck(lastSeqNo)) {
-           throw new IOException("BlockReader: error in packet header " +
-                                 header);
+        throw new IOException("BlockReader: error in packet header " +
+            header);
       }
 
       lastSeqNo = header.getSeqno();
@@ -263,7 +261,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       adjustChecksumBytes(header.getDataLen());
       if (header.getDataLen() > 0) {
         IOUtils.readFully(in, checksumBytes.array(), 0,
-                          checksumBytes.limit());
+            checksumBytes.limit());
       }
     }
 
@@ -284,14 +282,14 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       // How many chunks we can fit in databuffer
       //  - note this is a floor since we always read full chunks
       int chunksCanFit = Math.min(len / bytesPerChecksum,
-                                  checksumBuf.length / checksumSize);
+          checksumBuf.length / checksumSize);
 
       // How many chunks should we read
       checksumsToRead = Math.min(chunksLeft, chunksCanFit);
       // How many bytes should we actually read
       bytesToRead = Math.min(
-        checksumsToRead * bytesPerChecksum, // full chunks
-        dataLeft); // in case we have a partial
+          checksumsToRead * bytesPerChecksum, // full chunks
+          dataLeft); // in case we have a partial
     } else {
       // no checksum
       bytesToRead = Math.min(dataLeft, len);
@@ -328,7 +326,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       if (!hdr.isLastPacketInBlock() ||
           hdr.getDataLen() != 0) {
         throw new IOException("Expected empty end-of-read packet! Header: " +
-                              hdr);
+            hdr);
       }
 
       eos = true;
@@ -340,22 +338,22 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
     return bytesToRead;
   }
-  
+
   private RemoteBlockReader(String file, String bpid, long blockId,
       DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
     // Path is used only for printing block and file information in debug
     super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
-                    ":" + bpid + ":of:"+ file)/*too non path-like?*/,
-          1, verifyChecksum,
-          checksum.getChecksumSize() > 0? checksum : null, 
-          checksum.getBytesPerChecksum(),
-          checksum.getChecksumSize());
+            ":" + bpid + ":of:"+ file)/*too non path-like?*/,
+        1, verifyChecksum,
+        checksum.getChecksumSize() > 0? checksum : null,
+        checksum.getBytesPerChecksum(),
+        checksum.getChecksumSize());
 
     this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
         createSocketAddr(datanodeID.getXferAddr()));
-    
+
     this.peer = peer;
     this.datanodeID = datanodeID;
     this.in = in;
@@ -394,46 +392,46 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
    * @return New BlockReader instance, or null on error.
    */
   public static RemoteBlockReader newBlockReader(String file,
-                                     ExtendedBlock block, 
-                                     Token<BlockTokenIdentifier> blockToken,
-                                     long startOffset, long len,
-                                     int bufferSize, boolean verifyChecksum,
-                                     String clientName, Peer peer,
-                                     DatanodeID datanodeID,
-                                     PeerCache peerCache,
-                                     CachingStrategy cachingStrategy,
-                                     Tracer tracer)
-                                       throws IOException {
+      ExtendedBlock block,
+      Token<BlockTokenIdentifier> blockToken,
+      long startOffset, long len,
+      int bufferSize, boolean verifyChecksum,
+      String clientName, Peer peer,
+      DatanodeID datanodeID,
+      PeerCache peerCache,
+      CachingStrategy cachingStrategy,
+      Tracer tracer)
+      throws IOException {
     // in and out will be closed when sock is closed (by the caller)
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
         verifyChecksum, cachingStrategy);
-    
+
     //
     // Get bytes in block, set streams
     //
 
     DataInputStream in = new DataInputStream(
         new BufferedInputStream(peer.getInputStream(), bufferSize));
-    
+
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
         PBHelperClient.vintPrefixed(in));
     RemoteBlockReader2.checkSuccess(status, peer, block, file);
     ReadOpChecksumInfoProto checksumInfo =
-      status.getReadOpChecksumInfo();
+        status.getReadOpChecksumInfo();
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
         checksumInfo.getChecksum());
     //Warning when we get CHECKSUM_NULL?
-    
+
     // Read the first chunk offset.
     long firstChunkOffset = checksumInfo.getChunkOffset();
-    
+
     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
       throw new IOException("BlockReader: error in first chunk offset (" +
-                            firstChunkOffset + ") startOffset is " + 
-                            startOffset + " for file " + file);
+          firstChunkOffset + ") startOffset is " +
+          startOffset + " for file " + file);
     }
 
     return new RemoteBlockReader(file, block.getBlockPoolId(), block.getBlockId(),
@@ -453,7 +451,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
     // in will be closed when its Socket is closed.
   }
-  
+
   @Override
   public void readFully(byte[] buf, int readOffset, int amtToRead)
       throws IOException {
@@ -479,7 +477,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     } catch (IOException e) {
       // It's ok not to be able to send this. But something is probably wrong.
       LOG.info("Could not send read status (" + statusCode + ") to datanode " +
-               peer.getRemoteAddressString() + ": " + e.getMessage());
+          peer.getRemoteAddressString() + ": " + e.getMessage());
     }
   }
 
@@ -487,9 +485,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   public int read(ByteBuffer buf) throws IOException {
     throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
   }
-  
+
   @Override
-  public int available() throws IOException {
+  public int available() {
     // An optimistic estimate of how much data is available
     // to us without doing network I/O.
     return RemoteBlockReader2.TCP_WINDOW_SIZE;
@@ -499,7 +497,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   public boolean isLocal() {
     return isLocal;
   }
-  
+
   @Override
   public boolean isShortCircuit() {
     return false;

+ 62 - 64
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java

@@ -131,21 +131,19 @@ public class RemoteBlockReader2  implements BlockReader {
   public Peer getPeer() {
     return peer;
   }
-  
+
   @Override
-  public synchronized int read(byte[] buf, int off, int len) 
-                               throws IOException {
+  public synchronized int read(byte[] buf, int off, int len)
+      throws IOException {
     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(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")");
-      try {
+    if (curDataSlice == null ||
+        curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+      try (TraceScope ignored = tracer.newScope(
+          "RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
         readNextPacket();
-      } finally {
-        scope.close();
       }
     }
 
@@ -155,23 +153,21 @@ public class RemoteBlockReader2  implements BlockReader {
       // we're at EOF now
       return -1;
     }
-    
+
     int nRead = Math.min(curDataSlice.remaining(), len);
     curDataSlice.get(buf, off, nRead);
-    
+
     return nRead;
   }
 
 
   @Override
   public synchronized int read(ByteBuffer buf) throws IOException {
-    if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
-      TraceScope scope = tracer.newScope(
-          "RemoteBlockReader2#readNextPacket(" + blockId + ")");
-      try {
+    if (curDataSlice == null ||
+        (curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
+      try (TraceScope ignored = tracer.newScope(
+          "RemoteBlockReader2#readNextPacket(" + blockId + ")")) {
         readNextPacket();
-      } finally {
-        scope.close();
       }
     }
     if (curDataSlice.remaining() == 0) {
@@ -195,23 +191,24 @@ public class RemoteBlockReader2  implements BlockReader {
     PacketHeader curHeader = packetReceiver.getHeader();
     curDataSlice = packetReceiver.getDataSlice();
     assert curDataSlice.capacity() == curHeader.getDataLen();
-    
+
     LOG.trace("DFSClient readNextPacket got header {}", curHeader);
 
     // Sanity check the lengths
     if (!curHeader.sanityCheck(lastSeqNo)) {
-         throw new IOException("BlockReader: error in packet header " +
-                               curHeader);
+      throw new IOException("BlockReader: error in packet header " +
+          curHeader);
     }
-    
+
     if (curHeader.getDataLen() > 0) {
       int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum;
       int checksumsLen = chunks * checksumSize;
 
       assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
-        "checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() + 
-          " checksumsLen=" + checksumsLen;
-      
+          "checksum slice capacity=" +
+              packetReceiver.getChecksumSlice().capacity() +
+              " checksumsLen=" + checksumsLen;
+
       lastSeqNo = curHeader.getSeqno();
       if (verifyChecksum && curDataSlice.remaining() > 0) {
         // N.B.: the checksum error offset reported here is actually
@@ -223,8 +220,8 @@ public class RemoteBlockReader2  implements BlockReader {
             filename, curHeader.getOffsetInBlock());
       }
       bytesNeededToFinish -= curHeader.getDataLen();
-    }    
-    
+    }
+
     // First packet will include some data prior to the first byte
     // the user requested. Skip it.
     if (curHeader.getOffsetInBlock() < startOffset) {
@@ -243,7 +240,7 @@ public class RemoteBlockReader2  implements BlockReader {
       }
     }
   }
-  
+
   @Override
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
@@ -251,7 +248,8 @@ public class RemoteBlockReader2  implements BlockReader {
     long skipped = 0;
     while (skipped < n) {
       long needToSkip = n - skipped;
-      if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+      if (curDataSlice == null ||
+          curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
         readNextPacket();
       }
       if (curDataSlice.remaining() == 0) {
@@ -273,13 +271,13 @@ public class RemoteBlockReader2  implements BlockReader {
 
     PacketHeader trailer = packetReceiver.getHeader();
     if (!trailer.isLastPacketInBlock() ||
-       trailer.getDataLen() != 0) {
+        trailer.getDataLen() != 0) {
       throw new IOException("Expected empty end-of-read packet! Header: " +
-                            trailer);
+          trailer);
     }
   }
 
-  protected RemoteBlockReader2(String file, String bpid, long blockId,
+  protected RemoteBlockReader2(String file, long blockId,
       DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
@@ -320,7 +318,7 @@ public class RemoteBlockReader2  implements BlockReader {
 
     // in will be closed when its Socket is closed.
   }
-  
+
   /**
    * When the reader reaches end of the read, it sends a status response
    * (e.g. CHECKSUM_OK) to the DN. Failure to do so could lead to the DN
@@ -335,7 +333,7 @@ public class RemoteBlockReader2  implements BlockReader {
     } catch (IOException e) {
       // It's ok not to be able to send this. But something is probably wrong.
       LOG.info("Could not send read status (" + statusCode + ") to datanode " +
-               peer.getRemoteAddressString() + ": " + e.getMessage());
+          peer.getRemoteAddressString() + ": " + e.getMessage());
     }
   }
 
@@ -344,15 +342,15 @@ public class RemoteBlockReader2  implements BlockReader {
    */
   static void writeReadResult(OutputStream out, Status statusCode)
       throws IOException {
-    
+
     ClientReadStatusProto.newBuilder()
-      .setStatus(statusCode)
-      .build()
-      .writeDelimitedTo(out);
+        .setStatus(statusCode)
+        .build()
+        .writeDelimitedTo(out);
 
     out.flush();
   }
-  
+
   /**
    * File name to print when accessing a block directly (from servlets)
    * @param s Address of the block location
@@ -374,7 +372,7 @@ public class RemoteBlockReader2  implements BlockReader {
   public void readFully(byte[] buf, int off, int len) throws IOException {
     BlockReaderUtil.readFully(this, buf, off, len);
   }
-  
+
   /**
    * Create a new BlockReader specifically to satisfy a read.
    * This method also sends the OP_READ_BLOCK request.
@@ -391,18 +389,18 @@ public class RemoteBlockReader2  implements BlockReader {
    * @return New BlockReader instance, or null on error.
    */
   public static BlockReader newBlockReader(String file,
-                                     ExtendedBlock block,
-                                     Token<BlockTokenIdentifier> blockToken,
-                                     long startOffset, long len,
-                                     boolean verifyChecksum,
-                                     String clientName,
-                                     Peer peer, DatanodeID datanodeID,
-                                     PeerCache peerCache,
-                                     CachingStrategy cachingStrategy,
-                                     Tracer tracer) throws IOException {
+      ExtendedBlock block,
+      Token<BlockTokenIdentifier> blockToken,
+      long startOffset, long len,
+      boolean verifyChecksum,
+      String clientName,
+      Peer peer, DatanodeID datanodeID,
+      PeerCache peerCache,
+      CachingStrategy cachingStrategy,
+      Tracer tracer) throws IOException {
     // in and out will be closed when sock is closed (by the caller)
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-          peer.getOutputStream()));
+        peer.getOutputStream()));
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
         verifyChecksum, cachingStrategy);
 
@@ -415,7 +413,7 @@ public class RemoteBlockReader2  implements BlockReader {
         PBHelperClient.vintPrefixed(in));
     checkSuccess(status, peer, block, file);
     ReadOpChecksumInfoProto checksumInfo =
-      status.getReadOpChecksumInfo();
+        status.getReadOpChecksumInfo();
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
         checksumInfo.getChecksum());
     //Warning when we get CHECKSUM_NULL?
@@ -426,13 +424,13 @@ public class RemoteBlockReader2  implements BlockReader {
     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
       throw new IOException("BlockReader: error in first chunk offset (" +
-                            firstChunkOffset + ") startOffset is " +
-                            startOffset + " for file " + file);
+          firstChunkOffset + ") startOffset is " +
+          startOffset + " for file " + file);
     }
 
-    return new RemoteBlockReader2(file, block.getBlockPoolId(), block.getBlockId(),
-        checksum, verifyChecksum, startOffset, firstChunkOffset, len, peer,
-        datanodeID, peerCache, tracer);
+    return new RemoteBlockReader2(file, block.getBlockId(), checksum,
+        verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
+        peerCache, tracer);
   }
 
   static void checkSuccess(
@@ -440,26 +438,26 @@ public class RemoteBlockReader2  implements BlockReader {
       ExtendedBlock block, String file)
       throws IOException {
     String logInfo = "for OP_READ_BLOCK"
-      + ", self=" + peer.getLocalAddressString()
-      + ", remote=" + peer.getRemoteAddressString()
-      + ", for file " + file
-      + ", for pool " + block.getBlockPoolId()
-      + " block " + block.getBlockId() + "_" + block.getGenerationStamp();
+        + ", self=" + peer.getLocalAddressString()
+        + ", remote=" + peer.getRemoteAddressString()
+        + ", for file " + file
+        + ", for pool " + block.getBlockPoolId()
+        + " block " + block.getBlockId() + "_" + block.getGenerationStamp();
     DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
   }
-  
+
   @Override
-  public int available() throws IOException {
+  public int available() {
     // An optimistic estimate of how much data is available
     // to us without doing network I/O.
     return TCP_WINDOW_SIZE;
   }
-  
+
   @Override
   public boolean isLocal() {
     return isLocal;
   }
-  
+
   @Override
   public boolean isShortCircuit() {
     return false;

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

@@ -34,7 +34,7 @@ public interface RemotePeerFactory {
    * @param datanodeId    ID of destination DataNode
    * @return              A new Peer connected to the address.
    *
-   * @throws IOException  If there was an error connecting or creating 
+   * @throws IOException  If there was an error connecting or creating
    *                      the remote socket, encrypted stream, etc.
    */
   Peer newConnectedPeer(InetSocketAddress addr,

+ 19 - 20
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java

@@ -32,23 +32,23 @@ import com.google.common.collect.Maps;
 
 @InterfaceAudience.Private
 public class XAttrHelper {
-  
+
   /**
    * Build <code>XAttr</code> from xattr name with prefix.
    */
   public static XAttr buildXAttr(String name) {
     return buildXAttr(name, null);
   }
-  
+
   /**
    * Build <code>XAttr</code> from name with prefix and value.
-   * Name can not be null. Value can be null. The name and prefix 
+   * Name can not be null. Value can be null. The name and prefix
    * are validated.
    * Both name and namespace are case sensitive.
    */
   public static XAttr buildXAttr(String name, byte[] value) {
     Preconditions.checkNotNull(name, "XAttr name cannot be null.");
-    
+
     final int prefixIndex = name.indexOf(".");
     if (prefixIndex < 3) {// Prefix length is at least 3.
       throw new HadoopIllegalArgumentException("An XAttr name must be " +
@@ -56,7 +56,7 @@ public class XAttrHelper {
     } else if (prefixIndex == name.length() - 1) {
       throw new HadoopIllegalArgumentException("XAttr name cannot be empty.");
     }
-    
+
     NameSpace ns;
     final String prefix = name.substring(0, prefixIndex);
     if (StringUtils.equalsIgnoreCase(prefix, NameSpace.USER.toString())) {
@@ -77,12 +77,11 @@ public class XAttrHelper {
       throw new HadoopIllegalArgumentException("An XAttr name must be " +
           "prefixed with user/trusted/security/system/raw, followed by a '.'");
     }
-    XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
+
+    return (new XAttr.Builder()).setNameSpace(ns).setName(name.
         substring(prefixIndex + 1)).setValue(value).build();
-    
-    return xAttr;
   }
-  
+
   /**
    * Build xattr name with prefix as <code>XAttr</code> list.
    */
@@ -90,10 +89,10 @@ public class XAttrHelper {
     XAttr xAttr = buildXAttr(name);
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     xAttrs.add(xAttr);
-    
+
     return xAttrs;
   }
-  
+
   /**
    * Get value of first xattr from <code>XAttr</code> list
    */
@@ -108,7 +107,7 @@ public class XAttrHelper {
     }
     return value;
   }
-  
+
   /**
    * Get first xattr from <code>XAttr</code> list
    */
@@ -116,13 +115,13 @@ public class XAttrHelper {
     if (xAttrs != null && !xAttrs.isEmpty()) {
       return xAttrs.get(0);
     }
-    
+
     return null;
   }
-  
+
   /**
-   * Build xattr map from <code>XAttr</code> list, the key is 
-   * xattr name with prefix, and value is xattr value. 
+   * Build xattr map from <code>XAttr</code> list, the key is
+   * xattr name with prefix, and value is xattr value.
    */
   public static Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
     if (xAttrs == null) {
@@ -137,10 +136,10 @@ public class XAttrHelper {
       }
       xAttrMap.put(name, value);
     }
-    
+
     return xAttrMap;
   }
-  
+
   /**
    * Get name with prefix from <code>XAttr</code>
    */
@@ -164,11 +163,11 @@ public class XAttrHelper {
       throw new HadoopIllegalArgumentException("XAttr names can not be " +
           "null or empty.");
     }
-    
+
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
     for (String name : names) {
       xAttrs.add(buildXAttr(name, null));
     }
     return xAttrs;
-  } 
+  }
 }

+ 28 - 17
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java

@@ -31,7 +31,8 @@ public interface HdfsClientConfigKeys {
   long    DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
   String  DFS_REPLICATION_KEY = "dfs.replication";
   short   DFS_REPLICATION_DEFAULT = 3;
-  String  DFS_WEBHDFS_USER_PATTERN_KEY = "dfs.webhdfs.user.provider.user.pattern";
+  String  DFS_WEBHDFS_USER_PATTERN_KEY =
+      "dfs.webhdfs.user.provider.user.pattern";
   String  DFS_WEBHDFS_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
   String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       "^(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?(user|group|mask|other):[[A-Za-z_][A-Za-z0-9._-]]*:([rwx-]{3})?)*$";
@@ -125,7 +126,8 @@ public interface HdfsClientConfigKeys {
       "dfs.datanode.hdfs-blocks-metadata.enabled";
   boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
 
-  String  DFS_DATANODE_KERBEROS_PRINCIPAL_KEY = "dfs.datanode.kerberos.principal";
+  String  DFS_DATANODE_KERBEROS_PRINCIPAL_KEY =
+      "dfs.datanode.kerberos.principal";
   String  DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes";
   long    DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
   String  DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
@@ -142,7 +144,8 @@ public interface HdfsClientConfigKeys {
       "dfs.encrypt.data.transfer.cipher.key.bitlength";
   int    DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128;
 
-  String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS = "dfs.trustedchannel.resolver.class";
+  String DFS_TRUSTEDCHANNEL_RESOLVER_CLASS =
+      "dfs.trustedchannel.resolver.class";
 
   String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY =
       PREFIX + "replica.accessor.builder.classes";
@@ -161,13 +164,15 @@ public interface HdfsClientConfigKeys {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
 
     String  POLICY_ENABLED_KEY = PREFIX + "policy.enabled";
-    boolean POLICY_ENABLED_DEFAULT = false; 
+    boolean POLICY_ENABLED_DEFAULT = false;
     String  POLICY_SPEC_KEY = PREFIX + "policy.spec";
-    String  POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,... 
+    String  POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
 
-    String  TIMES_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "times.get-last-block-length";
+    String  TIMES_GET_LAST_BLOCK_LENGTH_KEY =
+        PREFIX + "times.get-last-block-length";
     int     TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT = 3;
-    String  INTERVAL_GET_LAST_BLOCK_LENGTH_KEY = PREFIX + "interval-ms.get-last-block-length";
+    String  INTERVAL_GET_LAST_BLOCK_LENGTH_KEY =
+        PREFIX + "interval-ms.get-last-block-length";
     int     INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
 
     String  MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
@@ -190,17 +195,19 @@ public interface HdfsClientConfigKeys {
     int     SLEEPTIME_MAX_DEFAULT = 15000;
     String  CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
     int     CONNECTION_RETRIES_DEFAULT = 0;
-    String  CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY = PREFIX + "connection.retries.on.timeouts";
+    String  CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_KEY =
+        PREFIX + "connection.retries.on.timeouts";
     int     CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
   }
-  
+
   /** dfs.client.write configuration properties */
   interface Write {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "write.";
 
     String  MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
     int     MAX_PACKETS_IN_FLIGHT_DEFAULT = 80;
-    String  EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY = PREFIX + "exclude.nodes.cache.expiry.interval.millis";
+    String  EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_KEY =
+        PREFIX + "exclude.nodes.cache.expiry.interval.millis";
     long    EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10*MINUTE;
 
     interface ByteArrayManager {
@@ -212,7 +219,8 @@ public interface HdfsClientConfigKeys {
       int     COUNT_THRESHOLD_DEFAULT = 128;
       String  COUNT_LIMIT_KEY = PREFIX + "count-limit";
       int     COUNT_LIMIT_DEFAULT = 2048;
-      String  COUNT_RESET_TIME_PERIOD_MS_KEY = PREFIX + "count-reset-time-period-ms";
+      String  COUNT_RESET_TIME_PERIOD_MS_KEY =
+          PREFIX + "count-reset-time-period-ms";
       long    COUNT_RESET_TIME_PERIOD_MS_DEFAULT = 10*SECOND;
     }
   }
@@ -223,9 +231,11 @@ public interface HdfsClientConfigKeys {
 
     String  RETRIES_KEY = PREFIX + "retries";
     int     RETRIES_DEFAULT = 3;
-    String  LOCATEFOLLOWINGBLOCK_RETRIES_KEY = PREFIX + "locateFollowingBlock.retries";
+    String  LOCATEFOLLOWINGBLOCK_RETRIES_KEY =
+        PREFIX + "locateFollowingBlock.retries";
     int     LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5;
-    String  LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY = PREFIX + "locateFollowingBlock.initial.delay.ms";
+    String  LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_KEY =
+        PREFIX + "locateFollowingBlock.initial.delay.ms";
     int     LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
 
     interface ReplaceDatanodeOnFailure {
@@ -243,8 +253,8 @@ public interface HdfsClientConfigKeys {
   /** dfs.client.read configuration properties */
   interface Read {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "read.";
-    
-    String  PREFETCH_SIZE_KEY = PREFIX + "prefetch.size"; 
+
+    String  PREFETCH_SIZE_KEY = PREFIX + "prefetch.size";
 
     interface ShortCircuit {
       String PREFIX = Read.PREFIX + "shortcircuit.";
@@ -267,7 +277,8 @@ public interface HdfsClientConfigKeys {
   interface ShortCircuit {
     String PREFIX = Read.PREFIX + "short.circuit.";
 
-    String  REPLICA_STALE_THRESHOLD_MS_KEY = PREFIX + "replica.stale.threshold.ms";
+    String  REPLICA_STALE_THRESHOLD_MS_KEY =
+        PREFIX + "replica.stale.threshold.ms";
     long    REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30*MINUTE;
   }
 
@@ -304,7 +315,7 @@ public interface HdfsClientConfigKeys {
     String  RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
     String  RETRY_MAX_ATTEMPTS_KEY = PREFIX + "retry.max.attempts";
     int     RETRY_MAX_ATTEMPTS_DEFAULT = 10;
-    
+
     // failover
     String  FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "failover.max.attempts";
     int     FAILOVER_MAX_ATTEMPTS_DEFAULT =  15;

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java

@@ -38,11 +38,11 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class HdfsDataInputStream extends FSDataInputStream {
-  public HdfsDataInputStream(DFSInputStream in) throws IOException {
+  public HdfsDataInputStream(DFSInputStream in) {
     super(in);
   }
 
-  public HdfsDataInputStream(CryptoInputStream in) throws IOException {
+  public HdfsDataInputStream(CryptoInputStream in) {
     super(in);
     Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
         "CryptoInputStream should wrap a DFSInputStream");
@@ -63,7 +63,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
    * @return the underlying output stream
    */
   public InputStream getWrappedStream() {
-      return in;
+    return in;
   }
 
   /**
@@ -90,10 +90,10 @@ public class HdfsDataInputStream extends FSDataInputStream {
   /**
    * Get the visible length of the file. It will include the length of the last
    * block even if that is in UnderConstruction state.
-   * 
+   *
    * @return The visible length of the file.
    */
-  public long getVisibleLength() throws IOException {
+  public long getVisibleLength() {
     return getDFSInputStream().getFileLength();
   }
 

+ 16 - 15
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java

@@ -41,33 +41,34 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
     super(out, stats, startPosition);
   }
 
-  public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
-      ) throws IOException {
+  public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats)
+      throws IOException {
     this(out, stats, 0L);
   }
 
-  public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
-      long startPosition) throws IOException {
+  public HdfsDataOutputStream(CryptoOutputStream out,
+      FileSystem.Statistics stats, long startPosition) throws IOException {
     super(out, stats, startPosition);
-    Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
+    Preconditions.checkArgument(
+        out.getWrappedStream() instanceof DFSOutputStream,
         "CryptoOutputStream should wrap a DFSOutputStream");
   }
 
-  public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
-      throws IOException {
+  public HdfsDataOutputStream(CryptoOutputStream out,
+      FileSystem.Statistics stats) throws IOException {
     this(out, stats, 0L);
   }
 
   /**
    * Get the actual number of replicas of the current block.
-   * 
+   *
    * This can be different from the designated replication factor of the file
    * because the namenode does not maintain replication for the blocks which are
    * currently being written to. Depending on the configuration, the client may
    * continue to write to a block even if a few datanodes in the write pipeline
    * have failed, or the client may add a new datanodes once a datanode has
    * failed.
-   * 
+   *
    * @return the number of valid replicas of the current block
    */
   public synchronized int getCurrentBlockReplication() throws IOException {
@@ -77,10 +78,10 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
     }
     return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
   }
-  
+
   /**
    * Sync buffered data to DataNodes (flush to disk devices).
-   * 
+   *
    * @param syncFlags
    *          Indicate the detailed semantic and actions of the hsync.
    * @throws IOException
@@ -89,13 +90,13 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
   public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
     OutputStream wrappedStream = getWrappedStream();
     if (wrappedStream instanceof CryptoOutputStream) {
-      ((CryptoOutputStream) wrappedStream).flush();
+      wrappedStream.flush();
       wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
     }
     ((DFSOutputStream) wrappedStream).hsync(syncFlags);
   }
-  
-  public static enum SyncFlag {
+
+  public enum SyncFlag {
 
     /**
      * When doing sync to DataNodes, also update the metadata (block length) in
@@ -107,6 +108,6 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
      * Sync the data to DataNode, close the current block, and allocate a new
      * block
      */
-    END_BLOCK;
+    END_BLOCK
   }
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java

@@ -84,13 +84,13 @@ public class CorruptFileBlockIterator implements RemoteIterator<Path> {
     }
   }
 
-  
+
   @Override
   public boolean hasNext() {
     return nextPath != null;
   }
 
-  
+
   @Override
   public Path next() throws IOException {
     if (!hasNext()) {
@@ -102,4 +102,4 @@ public class CorruptFileBlockIterator implements RemoteIterator<Path> {
 
     return result;
   }
-}
+}

+ 4 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java

@@ -79,7 +79,6 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Retry;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.ShortCircuit;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
 
-import java.lang.Class;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -254,9 +253,8 @@ public class DfsClientConf {
 
   @SuppressWarnings("unchecked")
   private List<Class<? extends ReplicaAccessorBuilder>>
-      loadReplicaAccessorBuilderClasses(Configuration conf)
-  {
-    String classNames[] = conf.getTrimmedStrings(
+      loadReplicaAccessorBuilderClasses(Configuration conf) {
+    String[] classNames = conf.getTrimmedStrings(
         HdfsClientConfigKeys.REPLICA_ACCESSOR_BUILDER_CLASSES_KEY);
     if (classNames.length == 0) {
       return Collections.emptyList();
@@ -267,8 +265,8 @@ public class DfsClientConf {
     for (String className: classNames) {
       try {
         Class<? extends ReplicaAccessorBuilder> cls =
-          (Class<? extends ReplicaAccessorBuilder>)
-            classLoader.loadClass(className);
+            (Class<? extends ReplicaAccessorBuilder>)
+                classLoader.loadClass(className);
         classes.add(cls);
       } catch (Throwable t) {
         LOG.warn("Unable to load " + className, t);

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

@@ -42,8 +42,8 @@ import org.slf4j.LoggerFactory;
 
 /**
  * <p>
- * Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing file-being-written leases
- * on the namenode.
+ * Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing
+ * file-being-written leases on the namenode.
  * When a file is opened for write (create or append),
  * namenode stores a file lease for recording the identity of the writer.
  * The writer (i.e. the DFSClient) is required to renew the lease periodically.
@@ -57,7 +57,8 @@ import org.slf4j.LoggerFactory;
  * <li>
  * It maintains a map from (namenode, user) pairs to lease renewers.
  * The same {@link LeaseRenewer} instance is used for renewing lease
- * for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and the same user.
+ * for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and
+ * the same user.
  * </li>
  * <li>
  * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
@@ -80,7 +81,7 @@ public class LeaseRenewer {
 
   /** Get a {@link LeaseRenewer} instance */
   public static LeaseRenewer getInstance(final String authority,
-      final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
+      final UserGroupInformation ugi, final DFSClient dfsc) {
     final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
     r.addClient(dfsc);
     return r;
@@ -136,7 +137,7 @@ public class LeaseRenewer {
     }
 
     /** A map for per user per namenode renewers. */
-    private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
+    private final Map<Key, LeaseRenewer> renewers = new HashMap<>();
 
     /** Get a renewer. */
     private synchronized LeaseRenewer get(final String authority,
@@ -189,7 +190,7 @@ public class LeaseRenewer {
   private final Factory.Key factorykey;
 
   /** A list of clients corresponding to this renewer. */
-  private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
+  private final List<DFSClient> dfsclients = new ArrayList<>();
 
   /**
    * A stringified stack trace of the call stack when the Lease Renewer
@@ -404,7 +405,7 @@ public class LeaseRenewer {
   private void renew() throws IOException {
     final List<DFSClient> copies;
     synchronized(this) {
-      copies = new ArrayList<DFSClient>(dfsclients);
+      copies = new ArrayList<>(dfsclients);
     }
     //sort the client names for finding out repeated names.
     Collections.sort(copies, new Comparator<DFSClient>() {
@@ -414,8 +415,7 @@ public class LeaseRenewer {
       }
     });
     String previousName = "";
-    for(int i = 0; i < copies.size(); i++) {
-      final DFSClient c = copies.get(i);
+    for (final DFSClient c : copies) {
       //skip if current client name is the same as the previous name.
       if (!c.getClientName().equals(previousName)) {
         if (!c.renewLease()) {
@@ -470,7 +470,7 @@ public class LeaseRenewer {
               LOG.debug("Lease renewer daemon for " + clientsString()
                   + " with renew id " + id + " is not current");
             } else {
-               LOG.debug("Lease renewer daemon for " + clientsString()
+              LOG.debug("Lease renewer daemon for " + clientsString()
                   + " with renew id " + id + " expired");
             }
           }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/package-info.java

@@ -15,4 +15,4 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.hdfs.client.impl;
+package org.apache.hadoop.hdfs.client.impl;

+ 27 - 22
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java

@@ -34,7 +34,7 @@ import java.util.List;
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public abstract class Event {
-  public static enum EventType {
+  public enum EventType {
     CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK, TRUNCATE
   }
 
@@ -98,8 +98,8 @@ public abstract class Event {
   @InterfaceAudience.Public
   public static class CreateEvent extends Event {
 
-    public static enum INodeType {
-      FILE, DIRECTORY, SYMLINK;
+    public enum INodeType {
+      FILE, DIRECTORY, SYMLINK
     }
 
     private INodeType iNodeType;
@@ -247,17 +247,21 @@ public abstract class Event {
     @InterfaceStability.Unstable
     public String toString() {
       StringBuilder content = new StringBuilder();
-      content.append("CreateEvent [INodeType=" + iNodeType + ", path=" + path
-          + ", ctime=" + ctime + ", replication=" + replication
-          + ", ownerName=" + ownerName + ", groupName=" + groupName
-          + ", perms=" + perms + ", ");
+      content.append("CreateEvent [INodeType=").append(iNodeType)
+          .append(", path=").append(path)
+          .append(", ctime=").append(ctime)
+          .append(", replication=").append(replication)
+          .append(", ownerName=").append(ownerName)
+          .append(", groupName=").append(groupName)
+          .append(", perms=").append(perms).append(", ");
 
       if (symlinkTarget != null) {
-        content.append("symlinkTarget=" + symlinkTarget + ", ");
+        content.append("symlinkTarget=").append(symlinkTarget).append(", ");
       }
 
-      content.append("overwrite=" + overwrite + ", defaultBlockSize="
-          + defaultBlockSize + "]");
+      content.append("overwrite=").append(overwrite)
+          .append(", defaultBlockSize=").append(defaultBlockSize)
+          .append("]");
       return content.toString();
     }
 
@@ -274,8 +278,8 @@ public abstract class Event {
   @InterfaceAudience.Public
   public static class MetadataUpdateEvent extends Event {
 
-    public static enum MetadataType {
-      TIMES, REPLICATION, OWNER, PERMS, ACLS, XATTRS;
+    public enum MetadataType {
+      TIMES, REPLICATION, OWNER, PERMS, ACLS, XATTRS
     }
 
     private String path;
@@ -434,28 +438,29 @@ public abstract class Event {
     @InterfaceStability.Unstable
     public String toString() {
       StringBuilder content = new StringBuilder();
-      content.append("MetadataUpdateEvent [path=" + path + ", metadataType="
-          + metadataType);
+      content.append("MetadataUpdateEvent [path=").append(path)
+          .append(", metadataType=").append(metadataType);
       switch (metadataType) {
       case TIMES:
-        content.append(", mtime=" + mtime + ", atime=" + atime);
+        content.append(", mtime=").append(mtime)
+            .append(", atime=").append(atime);
         break;
       case REPLICATION:
-        content.append(", replication=" + replication);
+        content.append(", replication=").append(replication);
         break;
       case OWNER:
-        content.append(", ownerName=" + ownerName
-            + ", groupName=" + groupName);
+        content.append(", ownerName=").append(ownerName)
+            .append(", groupName=").append(groupName);
         break;
       case PERMS:
-        content.append(", perms=" + perms);
+        content.append(", perms=").append(perms);
         break;
       case ACLS:
-        content.append(", acls=" + acls);
+        content.append(", acls=").append(acls);
         break;
       case XATTRS:
-        content.append(", xAttrs=" + xAttrs + ", xAttrsRemoved="
-            + xAttrsRemoved);
+        content.append(", xAttrs=").append(xAttrs)
+            .append(", xAttrsRemoved=").append(xAttrsRemoved);
         break;
       default:
         break;

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java

@@ -68,14 +68,14 @@ public class BasicInetPeer implements Peer {
 
   @Override
   public void setWriteTimeout(int timeoutMs) {
-   /* 
+   /*
     * We can't implement write timeouts. :(
-    * 
+    *
     * Java provides no facility to set a blocking write timeout on a Socket.
     * You can simulate a blocking write with a timeout by using
     * non-blocking I/O.  However, we can't use nio here, because this Socket
     * doesn't have an associated Channel.
-    * 
+    *
     * See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
     * more details.
     */
@@ -100,7 +100,7 @@ public class BasicInetPeer implements Peer {
   public String getLocalAddressString() {
     return socket.getLocalSocketAddress().toString();
   }
-  
+
   @Override
   public InputStream getInputStream() throws IOException {
     return in;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/DomainPeer.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.classification.InterfaceAudience;
 
 /**
- * Represents a peer that we communicate with by using blocking I/O 
+ * Represents a peer that we communicate with by using blocking I/O
  * on a UNIX domain socket.
  */
 @InterfaceAudience.Private

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java

@@ -38,12 +38,12 @@ public class EncryptedPeer implements Peer {
    * An encrypted InputStream.
    */
   private final InputStream in;
-  
+
   /**
    * An encrypted OutputStream.
    */
   private final OutputStream out;
-  
+
   /**
    * An encrypted ReadableByteChannel.
    */
@@ -53,7 +53,7 @@ public class EncryptedPeer implements Peer {
     this.enclosedPeer = enclosedPeer;
     this.in = ios.in;
     this.out = ios.out;
-    this.channel = ios.in instanceof ReadableByteChannel ? 
+    this.channel = ios.in instanceof ReadableByteChannel ?
         (ReadableByteChannel)ios.in : null;
   }
 

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.net.unix.DomainSocket;
 
 /**
- * Represents a peer that we communicate with by using non-blocking I/O 
+ * Represents a peer that we communicate with by using non-blocking I/O
  * on a Socket.
  */
 public class NioInetPeer implements Peer {
@@ -38,7 +38,7 @@ public class NioInetPeer implements Peer {
    * An InputStream which simulates blocking I/O with timeouts using NIO.
    */
   private final SocketInputStream in;
-  
+
   /**
    * An OutputStream which simulates blocking I/O with timeouts using NIO.
    */

+ 18 - 18
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java

@@ -34,24 +34,24 @@ public interface Peer extends Closeable {
    * @return                The input stream channel associated with this
    *                        peer, or null if it has none.
    */
-  public ReadableByteChannel getInputStreamChannel();
+  ReadableByteChannel getInputStreamChannel();
 
   /**
    * Set the read timeout on this peer.
    *
    * @param timeoutMs       The timeout in milliseconds.
    */
-  public void setReadTimeout(int timeoutMs) throws IOException;
+  void setReadTimeout(int timeoutMs) throws IOException;
 
   /**
    * @return                The receive buffer size.
    */
-  public int getReceiveBufferSize() throws IOException;
+  int getReceiveBufferSize() throws IOException;
 
   /**
    * @return                True if TCP_NODELAY is turned on.
    */
-  public boolean getTcpNoDelay() throws IOException;
+  boolean getTcpNoDelay() throws IOException;
 
   /**
    * Set the write timeout on this peer.
@@ -61,63 +61,63 @@ public interface Peer extends Closeable {
    *
    * @param timeoutMs       The timeout in milliseconds.
    */
-  public void setWriteTimeout(int timeoutMs) throws IOException;
+  void setWriteTimeout(int timeoutMs) throws IOException;
 
   /**
    * @return                true only if the peer is closed.
    */
-  public boolean isClosed();
-  
+  boolean isClosed();
+
   /**
    * Close the peer.
    *
    * It's safe to re-close a Peer that is already closed.
    */
-  public void close() throws IOException;
+  void close() throws IOException;
 
   /**
    * @return               A string representing the remote end of our
    *                       connection to the peer.
    */
-  public String getRemoteAddressString();
+  String getRemoteAddressString();
 
   /**
    * @return               A string representing the local end of our
    *                       connection to the peer.
    */
-  public String getLocalAddressString();
-  
+  String getLocalAddressString();
+
   /**
    * @return               An InputStream associated with the Peer.
    *                       This InputStream will be valid until you close
    *                       this peer with Peer#close.
    */
-  public InputStream getInputStream() throws IOException;
-  
+  InputStream getInputStream() throws IOException;
+
   /**
    * @return               An OutputStream associated with the Peer.
    *                       This OutputStream will be valid until you close
    *                       this peer with Peer#close.
    */
-  public OutputStream getOutputStream() throws IOException;
+  OutputStream getOutputStream() throws IOException;
 
   /**
    * @return               True if the peer resides on the same
    *                       computer as we.
    */
-  public boolean isLocal();
+  boolean isLocal();
 
   /**
    * @return               The DomainSocket associated with the current
    *                       peer, or null if there is none.
    */
-  public DomainSocket getDomainSocket();
-  
+  DomainSocket getDomainSocket();
+
   /**
    * Return true if the channel is secure.
    *
    * @return               True if our channel to this peer is not
    *                       susceptible to man-in-the-middle attacks.
    */
-  public boolean hasSecureChannel();
+  boolean hasSecureChannel();
 }

+ 13 - 18
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java

@@ -25,6 +25,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.*;
 
+import javax.annotation.Nonnull;
+
 /**************************************************
  * A Block is a Hadoop FS primitive, identified by a
  * long.
@@ -36,12 +38,10 @@ public class Block implements Writable, Comparable<Block> {
   public static final String BLOCK_FILE_PREFIX = "blk_";
   public static final String METADATA_EXTENSION = ".meta";
   static {                                      // register a ctor
-    WritableFactories.setFactory
-      (Block.class,
-       new WritableFactory() {
-         @Override
-         public Writable newInstance() { return new Block(); }
-       });
+    WritableFactories.setFactory(Block.class, new WritableFactory() {
+      @Override
+      public Writable newInstance() { return new Block(); }
+    });
   }
 
   public static final Pattern blockFilePattern = Pattern
@@ -208,20 +208,14 @@ public class Block implements Writable, Comparable<Block> {
   }
 
   @Override // Comparable
-  public int compareTo(Block b) {
+  public int compareTo(@Nonnull Block b) {
     return blockId < b.blockId ? -1 :
-           blockId > b.blockId ? 1 : 0;
+        blockId > b.blockId ? 1 : 0;
   }
 
   @Override // Object
   public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (!(o instanceof Block)) {
-      return false;
-    }
-    return compareTo((Block)o) == 0;
+    return this == o || o instanceof Block && compareTo((Block) o) == 0;
   }
 
   /**
@@ -230,9 +224,10 @@ public class Block implements Writable, Comparable<Block> {
    */
   public static boolean matchingIdAndGenStamp(Block a, Block b) {
     if (a == b) return true; // same block, or both null
-    if (a == null || b == null) return false; // only one null
-    return a.blockId == b.blockId &&
-           a.generationStamp == b.generationStamp;
+    // only one null
+    return !(a == null || b == null) &&
+        a.blockId == b.blockId &&
+        a.generationStamp == b.generationStamp;
   }
 
   @Override // Object

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java

@@ -33,7 +33,7 @@ public class BlockLocalPathInfo {
 
   /**
    * Constructs BlockLocalPathInfo.
-   * @param b The block corresponding to this lock path info. 
+   * @param b The block corresponding to this lock path info.
    * @param file Block data file.
    * @param metafile Metadata file for the block.
    */
@@ -48,12 +48,12 @@ public class BlockLocalPathInfo {
    * @return Block data file.
    */
   public String getBlockPath() {return localBlockPath;}
-  
+
   /**
    * @return the Block
    */
   public ExtendedBlock getBlock() { return block;}
-  
+
   /**
    * Get the Block metadata file.
    * @return Block metadata file.

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java

@@ -79,7 +79,7 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
    * @return a list of {@link StorageType}s for storing the replicas of a block.
    */
   public List<StorageType> chooseStorageTypes(final short replication) {
-    final List<StorageType> types = new LinkedList<StorageType>();
+    final List<StorageType> types = new LinkedList<>();
     int i = 0, j = 0;
 
     // Do not return transient storage types. We will not have accurate
@@ -136,11 +136,11 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
       final Iterable<StorageType> chosen,
       final EnumSet<StorageType> unavailables,
       final boolean isNewBlock) {
-    final List<StorageType> excess = new LinkedList<StorageType>();
+    final List<StorageType> excess = new LinkedList<>();
     final List<StorageType> storageTypes = chooseStorageTypes(
         replication, chosen, excess);
     final int expectedSize = storageTypes.size() - excess.size();
-    final List<StorageType> removed = new LinkedList<StorageType>();
+    final List<StorageType> removed = new LinkedList<>();
     for(int i = storageTypes.size() - 1; i >= 0; i--) {
       // replace/remove unavailable storage types.
       final StorageType t = storageTypes.get(i);
@@ -195,7 +195,7 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
   public List<StorageType> chooseExcess(final short replication,
       final Iterable<StorageType> chosen) {
     final List<StorageType> types = chooseStorageTypes(replication);
-    final List<StorageType> excess = new LinkedList<StorageType>();
+    final List<StorageType> excess = new LinkedList<>();
     diff(types, chosen, excess);
     return excess;
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java

@@ -42,4 +42,4 @@ public class CacheDirectiveEntry {
   public CacheDirectiveStats getStats() {
     return stats;
   }
-};
+}

+ 10 - 11
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java

@@ -46,7 +46,7 @@ public class CacheDirectiveInfo {
 
     /**
      * Builds a new CacheDirectiveInfo populated with the set properties.
-     * 
+     *
      * @return New CacheDirectiveInfo.
      */
     public CacheDirectiveInfo build() {
@@ -73,7 +73,7 @@ public class CacheDirectiveInfo {
 
     /**
      * Sets the id used in this request.
-     * 
+     *
      * @param id The id used in this request.
      * @return This builder, for call chaining.
      */
@@ -84,7 +84,7 @@ public class CacheDirectiveInfo {
 
     /**
      * Sets the path used in this request.
-     * 
+     *
      * @param path The path used in this request.
      * @return This builder, for call chaining.
      */
@@ -95,7 +95,7 @@ public class CacheDirectiveInfo {
 
     /**
      * Sets the replication used in this request.
-     * 
+     *
      * @param replication The replication used in this request.
      * @return This builder, for call chaining.
      */
@@ -106,7 +106,7 @@ public class CacheDirectiveInfo {
 
     /**
      * Sets the pool used in this request.
-     * 
+     *
      * @param pool The pool used in this request.
      * @return This builder, for call chaining.
      */
@@ -119,7 +119,7 @@ public class CacheDirectiveInfo {
      * Sets when the CacheDirective should expire. A
      * {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
      * relative expiration time.
-     * 
+     *
      * @param expiration when this CacheDirective should expire
      * @return This builder, for call chaining
      */
@@ -156,7 +156,7 @@ public class CacheDirectiveInfo {
      * <p>
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * expires.
-     * 
+     *
      * @param ms how long until the CacheDirective expires, in milliseconds
      * @return A relative Expiration
      */
@@ -169,7 +169,7 @@ public class CacheDirectiveInfo {
      * <p>
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * expires.
-     * 
+     *
      * @param date when the CacheDirective expires
      * @return An absolute Expiration
      */
@@ -182,7 +182,7 @@ public class CacheDirectiveInfo {
      * <p>
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * expires.
-     * 
+     *
      * @param ms when the CacheDirective expires, in milliseconds since the Unix
      *          epoch.
      * @return An absolute Expiration
@@ -350,9 +350,8 @@ public class CacheDirectiveInfo {
     }
     if (expiration != null) {
       builder.append(prefix).append("expiration: ").append(expiration);
-      prefix = ", ";
     }
     builder.append("}");
     return builder.toString();
   }
-};
+}

+ 4 - 7
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java

@@ -92,9 +92,8 @@ public class CacheDirectiveIterator
   @Override
   public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
       throws IOException {
-    BatchedEntries<CacheDirectiveEntry> entries = null;
-    TraceScope scope = tracer.newScope("listCacheDirectives");
-    try {
+    BatchedEntries<CacheDirectiveEntry> entries;
+    try (TraceScope ignored = tracer.newScope("listCacheDirectives")) {
       entries = namenode.listCacheDirectives(prevKey, filter);
     } catch (IOException e) {
       if (e.getMessage().contains("Filtering by ID is unsupported")) {
@@ -105,9 +104,9 @@ public class CacheDirectiveIterator
         // This is somewhat brittle, since it depends on directives being
         // returned in order of ascending ID.
         entries = namenode.listCacheDirectives(id - 1, filter);
-        for (int i=0; i<entries.size(); i++) {
+        for (int i = 0; i < entries.size(); i++) {
           CacheDirectiveEntry entry = entries.get(i);
-          if (entry.getInfo().getId().equals((Long)id)) {
+          if (entry.getInfo().getId().equals(id)) {
             return new SingleEntry(entry);
           }
         }
@@ -115,8 +114,6 @@ public class CacheDirectiveIterator
             "Did not find requested id " + id);
       }
       throw e;
-    } finally {
-      scope.close();
     }
     Preconditions.checkNotNull(entries);
     return entries;

+ 11 - 15
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java

@@ -35,7 +35,7 @@ public class CacheDirectiveStats {
 
     /**
      * Builds a new CacheDirectiveStats populated with the set properties.
-     * 
+     *
      * @return New CacheDirectiveStats.
      */
     public CacheDirectiveStats build() {
@@ -51,7 +51,7 @@ public class CacheDirectiveStats {
 
     /**
      * Sets the bytes needed by this directive.
-     * 
+     *
      * @param bytesNeeded The bytes needed.
      * @return This builder, for call chaining.
      */
@@ -62,7 +62,7 @@ public class CacheDirectiveStats {
 
     /**
      * Sets the bytes cached by this directive.
-     * 
+     *
      * @param bytesCached The bytes cached.
      * @return This builder, for call chaining.
      */
@@ -83,7 +83,7 @@ public class CacheDirectiveStats {
 
     /**
      * Sets the files cached by this directive.
-     * 
+     *
      * @param filesCached The number of files cached.
      * @return This builder, for call chaining.
      */
@@ -94,7 +94,7 @@ public class CacheDirectiveStats {
 
     /**
      * Sets whether this directive has expired.
-     * 
+     *
      * @param hasExpired if this directive has expired
      * @return This builder, for call chaining.
      */
@@ -156,14 +156,10 @@ public class CacheDirectiveStats {
 
   @Override
   public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append("{");
-    builder.append("bytesNeeded: ").append(bytesNeeded);
-    builder.append(", ").append("bytesCached: ").append(bytesCached);
-    builder.append(", ").append("filesNeeded: ").append(filesNeeded);
-    builder.append(", ").append("filesCached: ").append(filesCached);
-    builder.append(", ").append("hasExpired: ").append(hasExpired);
-    builder.append("}");
-    return builder.toString();
+    return "{" + "bytesNeeded: " + bytesNeeded + ", "
+        + "bytesCached: " + bytesCached + ", "
+        + "filesNeeded: " + filesNeeded + ", "
+        + "filesCached: " + filesCached + ", "
+        + "hasExpired: " + hasExpired + "}";
   }
-};
+}

+ 10 - 12
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java

@@ -145,7 +145,7 @@ public class CachePoolInfo {
   /**
    * Set the maximum relative expiration of directives of this pool in
    * milliseconds.
-   * 
+   *
    * @param ms in milliseconds
    * @return This builder, for call chaining.
    */
@@ -155,17 +155,15 @@ public class CachePoolInfo {
   }
 
   public String toString() {
-    return new StringBuilder().append("{").
-      append("poolName:").append(poolName).
-      append(", ownerName:").append(ownerName).
-      append(", groupName:").append(groupName).
-      append(", mode:").append((mode == null) ? "null" :
-          String.format("0%03o", mode.toShort())).
-      append(", limit:").append(limit).
-      append(", maxRelativeExpiryMs:").append(maxRelativeExpiryMs).
-      append("}").toString();
-  }
-  
+    return "{" + "poolName:" + poolName
+        + ", ownerName:" + ownerName
+        + ", groupName:" + groupName
+        + ", mode:"
+        + ((mode == null) ? "null" : String.format("0%03o", mode.toShort()))
+        + ", limit:" + limit
+        + ", maxRelativeExpiryMs:" + maxRelativeExpiryMs + "}";
+  }
+
   @Override
   public boolean equals(Object o) {
     if (o == null) { return false; }

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java

@@ -47,11 +47,8 @@ public class CachePoolIterator
   @Override
   public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
       throws IOException {
-    TraceScope scope = tracer.newScope("listCachePools");
-    try {
+    try (TraceScope ignored = tracer.newScope("listCachePools")) {
       return namenode.listCachePools(prevKey);
-    } finally {
-      scope.close();
     }
   }
 

+ 6 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java

@@ -66,7 +66,7 @@ public class CachePoolStats {
       return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
           filesNeeded, filesCached);
     }
-  };
+  }
 
   private final long bytesNeeded;
   private final long bytesCached;
@@ -104,12 +104,10 @@ public class CachePoolStats {
   }
 
   public String toString() {
-    return new StringBuilder().append("{").
-      append("bytesNeeded:").append(bytesNeeded).
-      append(", bytesCached:").append(bytesCached).
-      append(", bytesOverlimit:").append(bytesOverlimit).
-      append(", filesNeeded:").append(filesNeeded).
-      append(", filesCached:").append(filesCached).
-      append("}").toString();
+    return "{" + "bytesNeeded:" + bytesNeeded
+        + ", bytesCached:" + bytesCached
+        + ", bytesOverlimit:" + bytesOverlimit
+        + ", filesNeeded:" + filesNeeded
+        + ", filesCached:" + filesCached + "}";
   }
 }

+ 22 - 22
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -41,34 +41,34 @@ import org.apache.hadoop.security.token.TokenInfo;
 public interface ClientDatanodeProtocol {
   /**
    * Until version 9, this class ClientDatanodeProtocol served as both
-   * the client interface to the DN AND the RPC protocol used to 
+   * the client interface to the DN AND the RPC protocol used to
    * communicate with the NN.
-   * 
-   * This class is used by both the DFSClient and the 
+   *
+   * This class is used by both the DFSClient and the
    * DN server side to insulate from the protocol serialization.
-   * 
-   * If you are adding/changing DN's interface then you need to 
+   *
+   * If you are adding/changing DN's interface then you need to
    * change both this class and ALSO related protocol buffer
    * wire protocol definition in ClientDatanodeProtocol.proto.
-   * 
-   * For more details on protocol buffer wire protocol, please see 
+   *
+   * For more details on protocol buffer wire protocol, please see
    * .../org/apache/hadoop/hdfs/protocolPB/overview.html
-   * 
+   *
    * The log of historical changes can be retrieved from the svn).
    * 9: Added deleteBlockPool method
-   * 
+   *
    * 9 is the last version id when this class was used for protocols
-   *  serialization. DO not update this version any further. 
+   *  serialization. DO not update this version any further.
    */
-  public static final long versionID = 9L;
+  long versionID = 9L;
 
   /** Return the visible length of a replica. */
   long getReplicaVisibleLength(ExtendedBlock b) throws IOException;
-  
+
   /**
    * Refresh the list of federated namenodes from updated configuration
    * Adds new namenodes and stops the deleted namenodes.
-   * 
+   *
    * @throws IOException on error
    **/
   void refreshNamenodes() throws IOException;
@@ -76,19 +76,19 @@ public interface ClientDatanodeProtocol {
   /**
    * Delete the block pool directory. If force is false it is deleted only if
    * it is empty, otherwise it is deleted along with its contents.
-   * 
+   *
    * @param bpid Blockpool id to be deleted.
-   * @param force If false blockpool directory is deleted only if it is empty 
-   *          i.e. if it doesn't contain any block files, otherwise it is 
+   * @param force If false blockpool directory is deleted only if it is empty
+   *          i.e. if it doesn't contain any block files, otherwise it is
    *          deleted along with its contents.
    * @throws IOException
    */
   void deleteBlockPool(String bpid, boolean force) throws IOException;
-  
+
   /**
    * Retrieves the path names of the block file and metadata file stored on the
    * local file system.
-   * 
+   *
    * In order for this method to work, one of the following should be satisfied:
    * <ul>
    * <li>
@@ -98,7 +98,7 @@ public interface ClientDatanodeProtocol {
    * When security is enabled, kerberos authentication must be used to connect
    * to the datanode.</li>
    * </ul>
-   * 
+   *
    * @param block
    *          the specified block on the local datanode
    * @param token
@@ -109,7 +109,7 @@ public interface ClientDatanodeProtocol {
    */
   BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
       Token<BlockTokenIdentifier> token) throws IOException;
-  
+
   /**
    * Retrieves volume location information about a list of blocks on a datanode.
    * This is in the form of an opaque {@link org.apache.hadoop.fs.VolumeId}
@@ -136,9 +136,9 @@ public interface ClientDatanodeProtocol {
    *          down. The work includes advising clients to wait and saving
    *          certain states for quick restart. This should only be used when
    *          the stored data will remain the same during upgrade/restart.
-   * @throws IOException 
+   * @throws IOException
    */
-  void shutdownDatanode(boolean forUpgrade) throws IOException;  
+  void shutdownDatanode(boolean forUpgrade) throws IOException;
 
   /**
    * Obtains datanode info

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.Idempotent;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;

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

@@ -42,8 +42,10 @@ public class DSQuotaExceededException extends QuotaExceededException {
     String msg = super.getMessage();
     if (msg == null) {
       return "The DiskSpace quota" + (pathName==null?"": " of " + pathName)
-          + " is exceeded: quota = " + quota + " B = " + long2String(quota, "B", 2)
-          + " but diskspace consumed = " + count + " B = " + long2String(count, "B", 2);
+          + " is exceeded: quota = " + quota
+          + " B = " + long2String(quota, "B", 2)
+          + " but diskspace consumed = " + count
+          + " B = " + long2String(count, "B", 2);
     } else {
       return msg;
     }

+ 4 - 8
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java

@@ -233,14 +233,10 @@ public class DatanodeID implements Comparable<DatanodeID> {
 
   @Override
   public boolean equals(Object to) {
-    if (this == to) {
-      return true;
-    }
-    if (!(to instanceof DatanodeID)) {
-      return false;
-    }
-    return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
-        datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
+    return this == to ||
+        (to instanceof DatanodeID &&
+            getXferAddr().equals(((DatanodeID) to).getXferAddr()) &&
+            datanodeUuid.equals(((DatanodeID) to).getDatanodeUuid()));
   }
 
   @Override

+ 47 - 30
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -52,7 +52,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private int xceiverCount;
   private String location = NetworkTopology.DEFAULT_RACK;
   private String softwareVersion;
-  private List<String> dependentHostNames = new LinkedList<String>();
+  private List<String> dependentHostNames = new LinkedList<>();
   private String upgradeDomain;
 
   // Datanode administrative states
@@ -358,18 +358,18 @@ public class DatanodeInfo extends DatanodeID implements Node {
     float cacheRemainingPercent = getCacheRemainingPercent();
     String lookupName = NetUtils.getHostNameOfIP(getName());
 
-    buffer.append("Name: "+ getName());
+    buffer.append("Name: ").append(getName());
     if (lookupName != null) {
-      buffer.append(" (" + lookupName + ")");
+      buffer.append(" (").append(lookupName).append(")");
     }
     buffer.append("\n");
-    buffer.append("Hostname: " + getHostName() + "\n");
+    buffer.append("Hostname: ").append(getHostName()).append("\n");
 
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
-      buffer.append("Rack: "+location+"\n");
+      buffer.append("Rack: ").append(location).append("\n");
     }
     if (upgradeDomain != null) {
-      buffer.append("Upgrade domain: "+ upgradeDomain +"\n");
+      buffer.append("Upgrade domain: ").append(upgradeDomain).append("\n");
     }
     buffer.append("Decommission Status : ");
     if (isDecommissioned()) {
@@ -379,19 +379,30 @@ public class DatanodeInfo extends DatanodeID implements Node {
     } else {
       buffer.append("Normal\n");
     }
-    buffer.append("Configured Capacity: "+c+" ("+StringUtils.byteDesc(c)+")"+"\n");
-    buffer.append("DFS Used: "+u+" ("+StringUtils.byteDesc(u)+")"+"\n");
-    buffer.append("Non DFS Used: "+nonDFSUsed+" ("+StringUtils.byteDesc(nonDFSUsed)+")"+"\n");
-    buffer.append("DFS Remaining: " +r+ " ("+StringUtils.byteDesc(r)+")"+"\n");
-    buffer.append("DFS Used%: "+percent2String(usedPercent) + "\n");
-    buffer.append("DFS Remaining%: "+percent2String(remainingPercent) + "\n");
-    buffer.append("Configured Cache Capacity: "+cc+" ("+StringUtils.byteDesc(cc)+")"+"\n");
-    buffer.append("Cache Used: "+cu+" ("+StringUtils.byteDesc(cu)+")"+"\n");
-    buffer.append("Cache Remaining: " +cr+ " ("+StringUtils.byteDesc(cr)+")"+"\n");
-    buffer.append("Cache Used%: "+percent2String(cacheUsedPercent) + "\n");
-    buffer.append("Cache Remaining%: "+percent2String(cacheRemainingPercent) + "\n");
-    buffer.append("Xceivers: "+getXceiverCount()+"\n");
-    buffer.append("Last contact: "+new Date(lastUpdate)+"\n");
+    buffer.append("Configured Capacity: ").append(c).append(" (")
+        .append(StringUtils.byteDesc(c)).append(")").append("\n");
+    buffer.append("DFS Used: ").append(u).append(" (")
+        .append(StringUtils.byteDesc(u)).append(")").append("\n");
+    buffer.append("Non DFS Used: ").append(nonDFSUsed).append(" (")
+        .append(StringUtils.byteDesc(nonDFSUsed)).append(")").append("\n");
+    buffer.append("DFS Remaining: ").append(r).append(" (")
+        .append(StringUtils.byteDesc(r)).append(")").append("\n");
+    buffer.append("DFS Used%: ").append(percent2String(usedPercent))
+        .append("\n");
+    buffer.append("DFS Remaining%: ").append(percent2String(remainingPercent))
+        .append("\n");
+    buffer.append("Configured Cache Capacity: ").append(cc).append(" (")
+        .append(StringUtils.byteDesc(cc)).append(")").append("\n");
+    buffer.append("Cache Used: ").append(cu).append(" (")
+        .append(StringUtils.byteDesc(cu)).append(")").append("\n");
+    buffer.append("Cache Remaining: ").append(cr).append(" (")
+        .append(StringUtils.byteDesc(cr)).append(")").append("\n");
+    buffer.append("Cache Used%: ").append(percent2String(cacheUsedPercent))
+        .append("\n");
+    buffer.append("Cache Remaining%: ")
+        .append(percent2String(cacheRemainingPercent)).append("\n");
+    buffer.append("Xceivers: ").append(getXceiverCount()).append("\n");
+    buffer.append("Last contact: ").append(new Date(lastUpdate)).append("\n");
     return buffer.toString();
   }
 
@@ -408,10 +419,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
     float cacheUsedPercent = getCacheUsedPercent();
     buffer.append(getName());
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
-      buffer.append(" "+location);
+      buffer.append(" ").append(location);
     }
     if (upgradeDomain != null) {
-      buffer.append(" " + upgradeDomain);
+      buffer.append(" ").append(upgradeDomain);
     }
     if (isDecommissioned()) {
       buffer.append(" DD");
@@ -420,15 +431,21 @@ public class DatanodeInfo extends DatanodeID implements Node {
     } else {
       buffer.append(" IN");
     }
-    buffer.append(" " + c + "(" + StringUtils.byteDesc(c)+")");
-    buffer.append(" " + u + "(" + StringUtils.byteDesc(u)+")");
-    buffer.append(" " + percent2String(usedPercent));
-    buffer.append(" " + r + "(" + StringUtils.byteDesc(r)+")");
-    buffer.append(" " + cc + "(" + StringUtils.byteDesc(cc)+")");
-    buffer.append(" " + cu + "(" + StringUtils.byteDesc(cu)+")");
-    buffer.append(" " + percent2String(cacheUsedPercent));
-    buffer.append(" " + cr + "(" + StringUtils.byteDesc(cr)+")");
-    buffer.append(" " + new Date(lastUpdate));
+    buffer.append(" ").append(c).append("(").append(StringUtils.byteDesc(c))
+        .append(")");
+    buffer.append(" ").append(u).append("(").append(StringUtils.byteDesc(u))
+        .append(")");
+    buffer.append(" ").append(percent2String(usedPercent));
+    buffer.append(" ").append(r).append("(").append(StringUtils.byteDesc(r))
+        .append(")");
+    buffer.append(" ").append(cc).append("(").append(StringUtils.byteDesc(cc))
+        .append(")");
+    buffer.append(" ").append(cu).append("(").append(StringUtils.byteDesc(cu))
+        .append(")");
+    buffer.append(" ").append(percent2String(cacheUsedPercent));
+    buffer.append(" ").append(cr).append("(").append(StringUtils.byteDesc(cr))
+        .append(")");
+    buffer.append(" ").append(new Date(lastUpdate));
     return buffer.toString();
   }
 

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfoWithStorage.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.protocol;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -29,7 +28,7 @@ public class DatanodeInfoWithStorage extends DatanodeInfo {
   private final StorageType storageType;
 
   public DatanodeInfoWithStorage(DatanodeInfo from, String storageID,
-                                 StorageType storageType) {
+      StorageType storageType) {
     super(from);
     this.storageID = storageID;
     this.storageType = storageType;

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

@@ -55,10 +55,8 @@ public class DatanodeLocalInfo {
 
   /** A formatted string for printing the status of the DataNode. */
   public String getDatanodeLocalReport() {
-    StringBuilder buffer = new StringBuilder();
-    buffer.append("Uptime: " + getUptime());
-    buffer.append(", Software version: " + getSoftwareVersion());
-    buffer.append(", Config version: " + getConfigVersion());
-    return buffer.toString();
+    return ("Uptime: " + getUptime())
+        + ", Software version: " + getSoftwareVersion()
+        + ", Config version: " + getConfigVersion();
   }
 }

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java

@@ -39,7 +39,7 @@ public class EncryptionZoneIterator
   private final Tracer tracer;
 
   public EncryptionZoneIterator(ClientProtocol namenode, Tracer tracer) {
-    super(Long.valueOf(0));
+    super((long) 0);
     this.namenode = namenode;
     this.tracer = tracer;
   }
@@ -47,11 +47,8 @@ public class EncryptionZoneIterator
   @Override
   public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
       throws IOException {
-    TraceScope scope = tracer.newScope("listEncryptionZones");
-    try {
+    try (TraceScope ignored = tracer.newScope("listEncryptionZones")) {
       return namenode.listEncryptionZones(prevId);
-    } finally {
-      scope.close();
     }
   }
 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java

@@ -33,7 +33,8 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
 @InterfaceStability.Evolving
 public class HdfsFileStatus {
 
-  private final byte[] path;  // local name of the inode that's encoded in java UTF8
+  // local name of the inode that's encoded in java UTF8
+  private final byte[] path;
   private final byte[] symlink; // symlink target encoded in java UTF8 or null
   private final long length;
   private final boolean isdir;

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 
-/** 
+/**
  * Interface that represents the over the wire information
  * including block locations for a file.
  */
@@ -38,7 +38,7 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
 
   /**
    * Constructor
-   * 
+   *
    * @param length size
    * @param isdir if this is directory
    * @param block_replication the file's replication factor
@@ -49,7 +49,7 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
    * @param owner owner
    * @param group group
    * @param symlink symbolic link
-   * @param path local path name in java UTF8 format 
+   * @param path local path name in java UTF8 format
    * @param fileId the file id
    * @param locations block locations
    * @param feInfo file encryption info

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LastBlockWithStatus.java

@@ -31,7 +31,8 @@ public class LastBlockWithStatus {
 
   private final HdfsFileStatus fileStatus;
 
-  public LastBlockWithStatus(LocatedBlock lastBlock, HdfsFileStatus fileStatus) {
+  public LastBlockWithStatus(LocatedBlock lastBlock,
+      HdfsFileStatus fileStatus) {
     this.lastBlock = lastBlock;
     this.fileStatus = fileStatus;
   }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -65,13 +65,13 @@ public class LocatedBlock {
   }
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
-                      String[] storageIDs, StorageType[] storageTypes) {
+      String[] storageIDs, StorageType[] storageTypes) {
     this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
   }
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
-                      StorageType[] storageTypes, long startOffset,
-                      boolean corrupt, DatanodeInfo[] cachedLocs) {
+      StorageType[] storageTypes, long startOffset,
+      boolean corrupt, DatanodeInfo[] cachedLocs) {
     this.b = b;
     this.offset = startOffset;
     this.corrupt = corrupt;

+ 23 - 26
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java

@@ -32,7 +32,8 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
 @InterfaceStability.Evolving
 public class LocatedBlocks {
   private final long fileLength;
-  private final List<LocatedBlock> blocks; // array of blocks with prioritized locations
+  // array of blocks with prioritized locations
+  private final List<LocatedBlock> blocks;
   private final boolean underConstruction;
   private final LocatedBlock lastLocatedBlock;
   private final boolean isLastBlockComplete;
@@ -123,22 +124,22 @@ public class LocatedBlocks {
     key.setStartOffset(offset);
     key.getBlock().setNumBytes(1);
     Comparator<LocatedBlock> comp =
-      new Comparator<LocatedBlock>() {
-        // Returns 0 iff a is inside b or b is inside a
-        @Override
-        public int compare(LocatedBlock a, LocatedBlock b) {
-          long aBeg = a.getStartOffset();
-          long bBeg = b.getStartOffset();
-          long aEnd = aBeg + a.getBlockSize();
-          long bEnd = bBeg + b.getBlockSize();
-          if(aBeg <= bBeg && bEnd <= aEnd
-              || bBeg <= aBeg && aEnd <= bEnd)
-            return 0; // one of the blocks is inside the other
-          if(aBeg < bBeg)
-            return -1; // a's left bound is to the left of the b's
-          return 1;
-        }
-      };
+        new Comparator<LocatedBlock>() {
+          // Returns 0 iff a is inside b or b is inside a
+          @Override
+          public int compare(LocatedBlock a, LocatedBlock b) {
+            long aBeg = a.getStartOffset();
+            long bBeg = b.getStartOffset();
+            long aEnd = aBeg + a.getBlockSize();
+            long bEnd = bBeg + b.getBlockSize();
+            if(aBeg <= bBeg && bEnd <= aEnd
+                || bBeg <= aBeg && aEnd <= bEnd)
+              return 0; // one of the blocks is inside the other
+            if(aBeg < bBeg)
+              return -1; // a's left bound is to the left of the b's
+            return 1;
+          }
+        };
     return Collections.binarySearch(blocks, key, comp);
   }
 
@@ -176,14 +177,10 @@ public class LocatedBlocks {
 
   @Override
   public String toString() {
-    final StringBuilder b = new StringBuilder(getClass().getSimpleName());
-    b.append("{")
-     .append("\n  fileLength=").append(fileLength)
-     .append("\n  underConstruction=").append(underConstruction)
-     .append("\n  blocks=").append(blocks)
-     .append("\n  lastLocatedBlock=").append(lastLocatedBlock)
-     .append("\n  isLastBlockComplete=").append(isLastBlockComplete)
-     .append("}");
-    return b.toString();
+    return getClass().getSimpleName() + "{" + "\n  fileLength=" + fileLength
+        + "\n  underConstruction=" + underConstruction
+        + "\n  blocks=" + blocks
+        + "\n  lastLocatedBlock=" + lastLocatedBlock
+        + "\n  isLastBlockComplete=" + isLastBlockComplete + "}";
   }
 }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java

@@ -43,7 +43,7 @@ public final class NSQuotaExceededException extends QuotaExceededException {
     String msg = super.getMessage();
     if (msg == null) {
       msg = "The NameSpace quota (directories and files)" +
-      (pathName==null?"":(" of directory " + pathName)) +
+          (pathName==null?"":(" of directory " + pathName)) +
           " is exceeded: quota=" + quota + " file count=" + count;
 
       if (prefix != null) {

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

@@ -64,7 +64,6 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
 
   /**
    * Finalize the upgrade if not already finalized
-   * @param finalizeTime
    */
   public void finalize(long finalizeTime) {
     if (finalizeTime != 0) {
@@ -99,8 +98,11 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
   @Override
   public String toString() {
     return super.toString()
-      +  "\n     Start Time: " + (startTime == 0? "<NOT STARTED>": timestamp2String(startTime))
-      +  "\n  Finalize Time: " + (finalizeTime == 0? "<NOT FINALIZED>": timestamp2String(finalizeTime));
+        +  "\n     Start Time: "
+        + (startTime == 0 ? "<NOT STARTED>" : timestamp2String(startTime))
+        +  "\n  Finalize Time: "
+        + (finalizeTime == 0 ? "<NOT FINALIZED>" :
+        timestamp2String(finalizeTime));
   }
 
   private static String timestamp2String(long timestamp) {

+ 28 - 27
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java

@@ -27,7 +27,7 @@ import com.google.common.base.Objects;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 
 /**
- * This class represents to end users the difference between two snapshots of 
+ * This class represents to end users the difference between two snapshots of
  * the same directory, or the difference between a snapshot of the directory and
  * its current state. Instead of capturing all the details of the diff, this
  * class only lists where the changes happened and their types.
@@ -42,21 +42,21 @@ public class SnapshotDiffReport {
    * DELETE, and RENAME respectively.
    */
   public enum DiffType {
-    CREATE("+"),     
-    MODIFY("M"),    
-    DELETE("-"), 
+    CREATE("+"),
+    MODIFY("M"),
+    DELETE("-"),
     RENAME("R");
-    
+
     private final String label;
-    
-    private DiffType(String label) {
+
+    DiffType(String label) {
       this.label = label;
     }
-    
+
     public String getLabel() {
       return label;
     }
-    
+
     public static DiffType getTypeFromLabel(String label) {
       if (label.equals(CREATE.getLabel())) {
         return CREATE;
@@ -69,8 +69,8 @@ public class SnapshotDiffReport {
       }
       return null;
     }
-  };
-  
+  }
+
   /**
    * Representing the full path and diff type of a file/directory where changes
    * have happened.
@@ -98,7 +98,7 @@ public class SnapshotDiffReport {
       this.sourcePath = sourcePath;
       this.targetPath = targetPath;
     }
-    
+
     public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
         byte[][] targetPathComponents) {
       this.type = type;
@@ -106,7 +106,7 @@ public class SnapshotDiffReport {
       this.targetPath = targetPathComponents == null ? null : DFSUtilClient
           .byteArray2bytes(targetPathComponents);
     }
-    
+
     @Override
     public String toString() {
       String str = type.getLabel() + "\t" + getPathString(sourcePath);
@@ -115,7 +115,7 @@ public class SnapshotDiffReport {
       }
       return str;
     }
-    
+
     public DiffType getType() {
       return type;
     }
@@ -141,7 +141,7 @@ public class SnapshotDiffReport {
     public boolean equals(Object other) {
       if (this == other) {
         return true;
-      } 
+      }
       if (other != null && other instanceof DiffReportEntry) {
         DiffReportEntry entry = (DiffReportEntry) other;
         return type.equals(entry.getType())
@@ -150,25 +150,25 @@ public class SnapshotDiffReport {
       }
       return false;
     }
-    
+
     @Override
     public int hashCode() {
       return Objects.hashCode(getSourcePath(), getTargetPath());
     }
   }
-  
+
   /** snapshot root full path */
   private final String snapshotRoot;
 
   /** start point of the diff */
   private final String fromSnapshot;
-  
+
   /** end point of the diff */
   private final String toSnapshot;
-  
+
   /** list of diff */
   private final List<DiffReportEntry> diffList;
-  
+
   public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
       String toSnapshot, List<DiffReportEntry> entryList) {
     this.snapshotRoot = snapshotRoot;
@@ -177,7 +177,7 @@ public class SnapshotDiffReport {
     this.diffList = entryList != null ? entryList : Collections
         .<DiffReportEntry> emptyList();
   }
-  
+
   /** @return {@link #snapshotRoot}*/
   public String getSnapshotRoot() {
     return snapshotRoot;
@@ -192,23 +192,24 @@ public class SnapshotDiffReport {
   public String getLaterSnapshotName() {
     return toSnapshot;
   }
-  
+
   /** @return {@link #diffList} */
   public List<DiffReportEntry> getDiffList() {
     return diffList;
   }
-  
+
   @Override
   public String toString() {
     StringBuilder str = new StringBuilder();
-    String from = fromSnapshot == null || fromSnapshot.isEmpty() ? 
+    String from = fromSnapshot == null || fromSnapshot.isEmpty() ?
         "current directory" : "snapshot " + fromSnapshot;
     String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
         : "snapshot " + toSnapshot;
-    str.append("Difference between " + from + " and " + to
-        + " under directory " + snapshotRoot + ":" + LINE_SEPARATOR);
+    str.append("Difference between ").append(from).append(" and ").append(to)
+        .append(" under directory ").append(snapshotRoot).append(":")
+        .append(LINE_SEPARATOR);
     for (DiffReportEntry entry : diffList) {
-      str.append(entry.toString() + LINE_SEPARATOR);
+      str.append(entry.toString()).append(LINE_SEPARATOR);
     }
     return str.toString();
   }

+ 35 - 36
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java

@@ -31,30 +31,31 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
  */
 public class SnapshottableDirectoryStatus {
   /** Compare the statuses by full paths. */
-  public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR
-      = new Comparator<SnapshottableDirectoryStatus>() {
-    @Override
-    public int compare(SnapshottableDirectoryStatus left,
-                       SnapshottableDirectoryStatus right) {
-      int d = DFSUtilClient.compareBytes(left.parentFullPath, right.parentFullPath);
-      return d != 0? d
-          : DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
+  public static final Comparator<SnapshottableDirectoryStatus> COMPARATOR =
+      new Comparator<SnapshottableDirectoryStatus>() {
+        @Override
+        public int compare(SnapshottableDirectoryStatus left,
+            SnapshottableDirectoryStatus right) {
+          int d = DFSUtilClient.compareBytes(left.parentFullPath,
+              right.parentFullPath);
+          return d != 0? d
+              : DFSUtilClient.compareBytes(left.dirStatus.getLocalNameInBytes(),
               right.dirStatus.getLocalNameInBytes());
-    }
-  };
+        }
+      };
 
   /** Basic information of the snapshottable directory */
   private final HdfsFileStatus dirStatus;
-  
+
   /** Number of snapshots that have been taken*/
   private final int snapshotNumber;
-  
+
   /** Number of snapshots allowed. */
   private final int snapshotQuota;
-  
+
   /** Full path of the parent. */
   private final byte[] parentFullPath;
-  
+
   public SnapshottableDirectoryStatus(long modification_time, long access_time,
       FsPermission permission, String owner, String group, byte[] localName,
       long inodeId, int childrenNum,
@@ -80,7 +81,7 @@ public class SnapshottableDirectoryStatus {
   public int getSnapshotQuota() {
     return snapshotQuota;
   }
-  
+
   /**
    * @return Full path of the parent
    */
@@ -94,13 +95,13 @@ public class SnapshottableDirectoryStatus {
   public HdfsFileStatus getDirStatus() {
     return dirStatus;
   }
-  
+
   /**
    * @return Full path of the file
    */
   public Path getFullPath() {
-    String parentFullPathStr = 
-        (parentFullPath == null || parentFullPath.length == 0) ? 
+    String parentFullPathStr =
+        (parentFullPath == null || parentFullPath.length == 0) ?
             null : DFSUtilClient.bytes2String(parentFullPath);
     if (parentFullPathStr == null
         && dirStatus.getLocalNameInBytes().length == 0) {
@@ -111,13 +112,13 @@ public class SnapshottableDirectoryStatus {
           : new Path(parentFullPathStr, dirStatus.getLocalName());
     }
   }
-  
+
   /**
    * Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
    * @param stats The list of {@link SnapshottableDirectoryStatus}
    * @param out The given stream for printing.
    */
-  public static void print(SnapshottableDirectoryStatus[] stats, 
+  public static void print(SnapshottableDirectoryStatus[] stats,
       PrintStream out) {
     if (stats == null || stats.length == 0) {
       out.println();
@@ -133,30 +134,28 @@ public class SnapshottableDirectoryStatus {
       maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
       maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
     }
-    
-    StringBuilder fmt = new StringBuilder();
-    fmt.append("%s%s "); // permission string
-    fmt.append("%"  + maxRepl  + "s ");
-    fmt.append((maxOwner > 0) ? "%-" + maxOwner + "s " : "%s");
-    fmt.append((maxGroup > 0) ? "%-" + maxGroup + "s " : "%s");
-    fmt.append("%"  + maxLen   + "s ");
-    fmt.append("%s "); // mod time
-    fmt.append("%"  + maxSnapshotNum  + "s ");
-    fmt.append("%"  + maxSnapshotQuota  + "s ");
-    fmt.append("%s"); // path
-    
-    String lineFormat = fmt.toString();
+
+    String lineFormat = "%s%s " // permission string
+        + "%"  + maxRepl  + "s "
+        + (maxOwner > 0 ? "%-" + maxOwner + "s " : "%s")
+        + (maxGroup > 0 ? "%-" + maxGroup + "s " : "%s")
+        + "%"  + maxLen   + "s "
+        + "%s " // mod time
+        + "%"  + maxSnapshotNum  + "s "
+        + "%"  + maxSnapshotQuota  + "s "
+        + "%s"; // path
+
     SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
-         
+
     for (SnapshottableDirectoryStatus status : stats) {
-      String line = String.format(lineFormat, "d", 
+      String line = String.format(lineFormat, "d",
           status.dirStatus.getPermission(),
           status.dirStatus.getReplication(),
           status.dirStatus.getOwner(),
           status.dirStatus.getGroup(),
           String.valueOf(status.dirStatus.getLen()),
           dateFormat.format(new Date(status.dirStatus.getModificationTime())),
-          status.snapshotNumber, status.snapshotQuota, 
+          status.snapshotNumber, status.snapshotQuota,
           status.getFullPath().toString()
       );
       out.println(line);

+ 4 - 12
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java

@@ -18,14 +18,12 @@
 
 package org.apache.hadoop.hdfs.protocol;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.Path;
 
-/** 
+/**
  * Thrown when a symbolic link is encountered in a path.
  */
 @InterfaceAudience.Private
@@ -43,7 +41,7 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
   public UnresolvedPathException(String msg) {
     super(msg);
   }
-  
+
   public UnresolvedPathException(String path, String preceding,
       String remainder, String linkTarget) {
     this.path = path;
@@ -55,7 +53,7 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
   /**
    * Return a path with the link resolved with the target.
    */
-  public Path getResolvedPath() throws IOException {
+  public Path getResolvedPath() {
     // If the path is absolute we cam throw out the preceding part and
     // just append the remainder to the target, otherwise append each
     // piece to resolve the link in path.
@@ -76,12 +74,6 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
     if (msg != null) {
       return msg;
     }
-    String myMsg = "Unresolved path " + path;
-    try {
-      return getResolvedPath().toString();
-    } catch (IOException e) {
-      // Ignore
-    }
-    return myMsg;
+    return getResolvedPath().toString();
   }
 }

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

@@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 public enum BlockConstructionStage {
   /** The enumerates are always listed as regular stage followed by the
-   * recovery stage. 
+   * recovery stage.
    * Changing this order will make getRecoveryStage not working.
    */
   // pipeline set up for block append
@@ -46,9 +46,9 @@ public enum BlockConstructionStage {
   TRANSFER_RBW,
   // transfer Finalized for adding datanodes
   TRANSFER_FINALIZED;
-  
+
   final static private byte RECOVERY_BIT = (byte)1;
-  
+
   /**
    * get the recovery stage of this stage
    */
@@ -59,4 +59,4 @@ public enum BlockConstructionStage {
       return values()[ordinal()|RECOVERY_BIT];
     }
   }
-}    
+}

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java

@@ -65,7 +65,9 @@ public abstract class DataTransferProtoUtil {
   }
 
   public static DataChecksum fromProto(ChecksumProto proto) {
-    if (proto == null) return null;
+    if (proto == null) {
+      return null;
+    }
 
     int bytesPerChecksum = proto.getBytesPerChecksum();
     DataChecksum.Type type = PBHelperClient.convert(proto.getType());
@@ -74,19 +76,17 @@ public abstract class DataTransferProtoUtil {
 
   static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk,
       String client, Token<BlockTokenIdentifier> blockToken) {
-    ClientOperationHeaderProto header =
-      ClientOperationHeaderProto.newBuilder()
-        .setBaseHeader(buildBaseHeader(blk, blockToken))
-        .setClientName(client)
-        .build();
-    return header;
+    return ClientOperationHeaderProto.newBuilder()
+      .setBaseHeader(buildBaseHeader(blk, blockToken))
+      .setClientName(client)
+      .build();
   }
 
   static BaseHeaderProto buildBaseHeader(ExtendedBlock blk,
       Token<BlockTokenIdentifier> blockToken) {
     BaseHeaderProto.Builder builder =  BaseHeaderProto.newBuilder()
-      .setBlock(PBHelperClient.convert(blk))
-      .setToken(PBHelperClient.convert(blockToken));
+        .setBlock(PBHelperClient.convert(blk))
+        .setToken(PBHelperClient.convert(blockToken));
     SpanId spanId = Tracer.getCurrentSpanId();
     if (spanId.isValid()) {
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()

+ 28 - 28
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java

@@ -39,21 +39,21 @@ import org.slf4j.LoggerFactory;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public interface DataTransferProtocol {
-  public static final Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
-  
+  Logger LOG = LoggerFactory.getLogger(DataTransferProtocol.class);
+
   /** Version for data transfers between clients and datanodes
    * This should change when serialization of DatanodeInfo, not just
-   * when protocol changes. It is not very obvious. 
+   * when protocol changes. It is not very obvious.
    */
   /*
    * Version 28:
    *    Declare methods in DataTransferProtocol interface.
    */
-  public static final int DATA_TRANSFER_VERSION = 28;
+  int DATA_TRANSFER_VERSION = 28;
 
-  /** 
+  /**
    * Read a block.
-   * 
+   *
    * @param blk the block being read.
    * @param blockToken security token for accessing the block.
    * @param clientName client's name.
@@ -63,7 +63,7 @@ public interface DataTransferProtocol {
    *        checksums
    * @param cachingStrategy  The caching strategy to use.
    */
-  public void readBlock(final ExtendedBlock blk,
+  void readBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final long blockOffset,
@@ -77,7 +77,7 @@ public interface DataTransferProtocol {
    * The other downstream datanodes are specified by the targets parameter.
    * Note that the receiver {@link DatanodeInfo} is not required in the
    * parameter list since the receiver datanode knows its info.  However, the
-   * {@link StorageType} for storing the replica in the receiver datanode is a 
+   * {@link StorageType} for storing the replica in the receiver datanode is a
    * parameter since the receiver datanode may support multiple storage types.
    *
    * @param blk the block being written.
@@ -96,12 +96,12 @@ public interface DataTransferProtocol {
    * @param pinning whether to pin the block, so Balancer won't move it.
    * @param targetPinnings whether to pin the block on target datanode
    */
-  public void writeBlock(final ExtendedBlock blk,
-      final StorageType storageType, 
+  void writeBlock(final ExtendedBlock blk,
+      final StorageType storageType,
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final DatanodeInfo[] targets,
-      final StorageType[] targetStorageTypes, 
+      final StorageType[] targetStorageTypes,
       final DatanodeInfo source,
       final BlockConstructionStage stage,
       final int pipelineSize,
@@ -118,13 +118,13 @@ public interface DataTransferProtocol {
    * The block stage must be
    * either {@link BlockConstructionStage#TRANSFER_RBW}
    * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
-   * 
+   *
    * @param blk the block being transferred.
    * @param blockToken security token for accessing the block.
    * @param clientName client's name.
    * @param targets target datanodes.
    */
-  public void transferBlock(final ExtendedBlock blk,
+  void transferBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final DatanodeInfo[] targets,
@@ -135,14 +135,14 @@ public interface DataTransferProtocol {
    *
    * @param blk             The block to get file descriptors for.
    * @param blockToken      Security token for accessing the block.
-   * @param slotId          The shared memory slot id to use, or null 
+   * @param slotId          The shared memory slot id to use, or null
    *                          to use no slot id.
-   * @param maxVersion      Maximum version of the block data the client 
+   * @param maxVersion      Maximum version of the block data the client
    *                          can understand.
    * @param supportsReceiptVerification  True if the client supports
    *                          receipt verification.
    */
-  public void requestShortCircuitFds(final ExtendedBlock blk,
+  void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
         throws IOException;
@@ -152,51 +152,51 @@ public interface DataTransferProtocol {
    *
    * @param slotId          SlotID used by the earlier file descriptors.
    */
-  public void releaseShortCircuitFds(final SlotId slotId) throws IOException;
+  void releaseShortCircuitFds(final SlotId slotId) throws IOException;
 
   /**
    * Request a short circuit shared memory area from a DataNode.
-   * 
+   *
    * @param clientName       The name of the client.
    */
-  public void requestShortCircuitShm(String clientName) throws IOException;
-  
+  void requestShortCircuitShm(String clientName) throws IOException;
+
   /**
    * Receive a block from a source datanode
    * and then notifies the namenode
    * to remove the copy from the original datanode.
    * Note that the source datanode and the original datanode can be different.
    * It is used for balancing purpose.
-   * 
+   *
    * @param blk the block being replaced.
    * @param storageType the {@link StorageType} for storing the block.
    * @param blockToken security token for accessing the block.
    * @param delHint the hint for deleting the block in the original datanode.
    * @param source the source datanode for receiving the block.
    */
-  public void replaceBlock(final ExtendedBlock blk,
-      final StorageType storageType, 
+  void replaceBlock(final ExtendedBlock blk,
+      final StorageType storageType,
       final Token<BlockTokenIdentifier> blockToken,
       final String delHint,
       final DatanodeInfo source) throws IOException;
 
   /**
-   * Copy a block. 
+   * Copy a block.
    * It is used for balancing purpose.
-   * 
+   *
    * @param blk the block being copied.
    * @param blockToken security token for accessing the block.
    */
-  public void copyBlock(final ExtendedBlock blk,
+  void copyBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken) throws IOException;
 
   /**
    * Get block checksum (MD5 of CRC32).
-   * 
+   *
    * @param blk a block.
    * @param blockToken security token for accessing the block.
    * @throws IOException
    */
-  public void blockChecksum(final ExtendedBlock blk,
+  void blockChecksum(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken) throws IOException;
 }

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

@@ -29,9 +29,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 public class IOStreamPair {
   public final InputStream in;
   public final OutputStream out;
-  
+
   public IOStreamPair(InputStream in, OutputStream out) {
     this.in = in;
     this.out = out;
   }
-}
+}

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

@@ -42,11 +42,11 @@ public enum Op {
 
   /** The code for this operation. */
   public final byte code;
-  
-  private Op(byte code) {
+
+  Op(byte code) {
     this.code = code;
   }
-  
+
   private static final int FIRST_CODE = values()[0].code;
   /** Return the object represented by the code. */
   private static Op valueOf(byte code) {

+ 20 - 22
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java

@@ -36,14 +36,14 @@ import com.google.protobuf.InvalidProtocolBufferException;
  * Header data for each packet that goes through the read/write pipelines.
  * Includes all of the information about the packet, excluding checksums and
  * actual data.
- * 
+ *
  * This data includes:
  *  - the offset in bytes into the HDFS block of the data in this packet
  *  - the sequence number of this packet in the pipeline
  *  - whether or not this is the last packet in the pipeline
  *  - the length of the data in this packet
  *  - whether or not this packet should be synced by the DNs.
- *  
+ *
  * When serialized, this header is written out as a protocol buffer, preceded
  * by a 4-byte integer representing the full packet length, and a 2-byte short
  * representing the header length.
@@ -51,8 +51,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class PacketHeader {
-  private static final int MAX_PROTO_SIZE = 
-    PacketHeaderProto.newBuilder()
+  private static final int MAX_PROTO_SIZE = PacketHeaderProto.newBuilder()
       .setOffsetInBlock(0)
       .setSeqno(0)
       .setLastPacketInBlock(false)
@@ -76,21 +75,21 @@ public class PacketHeader {
     Preconditions.checkArgument(packetLen >= Ints.BYTES,
         "packet len %s should always be at least 4 bytes",
         packetLen);
-    
+
     PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
-      .setOffsetInBlock(offsetInBlock)
-      .setSeqno(seqno)
-      .setLastPacketInBlock(lastPacketInBlock)
-      .setDataLen(dataLen);
-      
+        .setOffsetInBlock(offsetInBlock)
+        .setSeqno(seqno)
+        .setLastPacketInBlock(lastPacketInBlock)
+        .setDataLen(dataLen);
+
     if (syncBlock) {
       // Only set syncBlock if it is specified.
       // This is wire-incompatible with Hadoop 2.0.0-alpha due to HDFS-3721
       // because it changes the length of the packet header, and BlockReceiver
       // in that version did not support variable-length headers.
-      builder.setSyncBlock(syncBlock);
+      builder.setSyncBlock(true);
     }
-      
+
     proto = builder.build();
   }
 
@@ -121,16 +120,16 @@ public class PacketHeader {
   @Override
   public String toString() {
     return "PacketHeader with packetLen=" + packetLen +
-      " header data: " + 
+      " header data: " +
       proto.toString();
   }
-  
+
   public void setFieldsFromData(
       int packetLen, byte[] headerData) throws InvalidProtocolBufferException {
     this.packetLen = packetLen;
     proto = PacketHeaderProto.parseFrom(headerData);
   }
-  
+
   public void readFields(ByteBuffer buf) throws IOException {
     packetLen = buf.getInt();
     short protoLen = buf.getShort();
@@ -138,7 +137,7 @@ public class PacketHeader {
     buf.get(data);
     proto = PacketHeaderProto.parseFrom(data);
   }
-  
+
   public void readFields(DataInputStream in) throws IOException {
     this.packetLen = in.readInt();
     short protoLen = in.readShort();
@@ -170,7 +169,7 @@ public class PacketHeader {
       throw new RuntimeException(e);
     }
   }
-  
+
   public void write(DataOutputStream out) throws IOException {
     assert proto.getSerializedSize() <= MAX_PROTO_SIZE
     : "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
@@ -178,7 +177,7 @@ public class PacketHeader {
     out.writeShort(proto.getSerializedSize());
     proto.writeTo(out);
   }
-  
+
   public byte[] getBytes() {
     ByteBuffer buf = ByteBuffer.allocate(getSerializedSize());
     putInBuffer(buf);
@@ -187,8 +186,8 @@ public class PacketHeader {
 
   /**
    * Perform a sanity check on the packet, returning true if it is sane.
-   * @param lastSeqNo the previous sequence number received - we expect the current
-   * sequence number to be larger by 1.
+   * @param lastSeqNo the previous sequence number received - we expect the
+   *                  current sequence number to be larger by 1.
    */
   public boolean sanityCheck(long lastSeqNo) {
     // We should only have a non-positive data length for the last packet
@@ -196,8 +195,7 @@ public class PacketHeader {
     // The last packet should not contain data
     if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
     // Seqnos should always increase by 1 with each packet received
-    if (proto.getSeqno() != lastSeqNo + 1) return false;
-    return true;
+    return proto.getSeqno() == lastSeqNo + 1;
   }
 
   @Override

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

@@ -48,7 +48,7 @@ public class PacketReceiver implements Closeable {
   private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024;
 
   static final Logger LOG = LoggerFactory.getLogger(PacketReceiver.class);
-  
+
   private static final DirectBufferPool bufferPool = new DirectBufferPool();
   private final boolean useDirectBuffers;
 
@@ -58,12 +58,12 @@ public class PacketReceiver implements Closeable {
    * length prefixes.
    */
   private ByteBuffer curPacketBuf = null;
-  
+
   /**
    * A slice of {@link #curPacketBuf} which contains just the checksums.
    */
   private ByteBuffer curChecksumSlice = null;
-  
+
   /**
    * A slice of {@link #curPacketBuf} which contains just the data.
    */
@@ -73,7 +73,7 @@ public class PacketReceiver implements Closeable {
    * The packet header of the most recently read packet.
    */
   private PacketHeader curHeader;
-  
+
   public PacketReceiver(boolean useDirectBuffers) {
     this.useDirectBuffers = useDirectBuffers;
     reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN);
@@ -86,14 +86,14 @@ public class PacketReceiver implements Closeable {
   public ByteBuffer getDataSlice() {
     return curDataSlice;
   }
-  
+
   public ByteBuffer getChecksumSlice() {
     return curChecksumSlice;
   }
 
   /**
    * Reads all of the data for the next packet into the appropriate buffers.
-   * 
+   *
    * The data slice and checksum slice members will be set to point to the
    * user data and corresponding checksums. The header will be parsed and
    * set.
@@ -134,7 +134,7 @@ public class PacketReceiver implements Closeable {
     doReadFully(ch, in, curPacketBuf);
     curPacketBuf.flip();
     int payloadLen = curPacketBuf.getInt();
-    
+
     if (payloadLen < Ints.BYTES) {
       // The "payload length" includes its own length. Therefore it
       // should never be less than 4 bytes
@@ -146,7 +146,7 @@ public class PacketReceiver implements Closeable {
     if (headerLen < 0) {
       throw new IOException("Invalid header length " + headerLen);
     }
-    
+
     LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
         dataPlusChecksumLen, headerLen);
 
@@ -177,18 +177,18 @@ public class PacketReceiver implements Closeable {
       curHeader = new PacketHeader();
     }
     curHeader.setFieldsFromData(payloadLen, headerBuf);
-    
+
     // Compute the sub-slices of the packet
     int checksumLen = dataPlusChecksumLen - curHeader.getDataLen();
     if (checksumLen < 0) {
-      throw new IOException("Invalid packet: data length in packet header " + 
+      throw new IOException("Invalid packet: data length in packet header " +
           "exceeds data length received. dataPlusChecksumLen=" +
-          dataPlusChecksumLen + " header: " + curHeader); 
+          dataPlusChecksumLen + " header: " + curHeader);
     }
-    
+
     reslicePacket(headerLen, checksumLen, curHeader.getDataLen());
   }
-  
+
   /**
    * Rewrite the last-read packet on the wire to the given output stream.
    */
@@ -200,7 +200,7 @@ public class PacketReceiver implements Closeable {
         curPacketBuf.remaining());
   }
 
-  
+
   private static void doReadFully(ReadableByteChannel ch, InputStream in,
       ByteBuffer buf) throws IOException {
     if (ch != null) {
@@ -222,7 +222,7 @@ public class PacketReceiver implements Closeable {
     //   32-bit  16-bit   <protobuf>  <variable length>
     //   |--- lenThroughHeader ----|
     //   |----------- lenThroughChecksums   ----|
-    //   |------------------- lenThroughData    ------| 
+    //   |------------------- lenThroughData    ------|
     int lenThroughHeader = PacketHeader.PKT_LENGTHS_LEN + headerLen;
     int lenThroughChecksums = lenThroughHeader + checksumsLen;
     int lenThroughData = lenThroughChecksums + dataLen;
@@ -242,14 +242,14 @@ public class PacketReceiver implements Closeable {
     curPacketBuf.position(lenThroughChecksums);
     curPacketBuf.limit(lenThroughData);
     curDataSlice = curPacketBuf.slice();
-    
+
     // Reset buffer to point to the entirety of the packet (including
     // length prefixes)
     curPacketBuf.position(0);
     curPacketBuf.limit(lenThroughData);
   }
 
-  
+
   private static void readChannelFully(ReadableByteChannel ch, ByteBuffer buf)
       throws IOException {
     while (buf.remaining() > 0) {
@@ -259,7 +259,7 @@ public class PacketReceiver implements Closeable {
       }
     }
   }
-  
+
   private void reallocPacketBuf(int atLeastCapacity) {
     // Realloc the buffer if this packet is longer than the previous
     // one.
@@ -277,12 +277,12 @@ public class PacketReceiver implements Closeable {
         curPacketBuf.flip();
         newBuf.put(curPacketBuf);
       }
-      
+
       returnPacketBufToPool();
       curPacketBuf = newBuf;
     }
   }
-  
+
   private void returnPacketBufToPool() {
     if (curPacketBuf != null && curPacketBuf.isDirect()) {
       bufferPool.returnBuffer(curPacketBuf);
@@ -294,7 +294,7 @@ public class PacketReceiver implements Closeable {
   public void close() {
     returnPacketBufToPool();
   }
-  
+
   @Override
   protected void finalize() throws Throwable {
     try {

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java

@@ -94,7 +94,7 @@ public class PipelineAck {
   /** default constructor **/
   public PipelineAck() {
   }
-  
+
   /**
    * Constructor assuming no next DN in pipeline
    * @param seqno sequence number
@@ -125,7 +125,7 @@ public class PipelineAck {
       .setDownstreamAckTimeNanos(downstreamAckTimeNanos)
       .build();
   }
-  
+
   /**
    * Get the sequence number
    * @return the sequence number
@@ -133,7 +133,7 @@ public class PipelineAck {
   public long getSeqno() {
     return proto.getSeqno();
   }
-  
+
   /**
    * Get the number of replies
    * @return the number of replies
@@ -141,7 +141,7 @@ public class PipelineAck {
   public short getNumOfReplies() {
     return (short)proto.getReplyCount();
   }
-  
+
   /**
    * get the header flag of ith reply
    */
@@ -179,7 +179,7 @@ public class PipelineAck {
   }
 
   /**
-   * Returns the OOB status if this ack contains one. 
+   * Returns the OOB status if this ack contains one.
    * @return null if it is not an OOB ack.
    */
   public Status getOOBStatus() {
@@ -216,7 +216,7 @@ public class PipelineAck {
   public void write(OutputStream out) throws IOException {
     proto.writeDelimitedTo(out);
   }
-  
+
   @Override //Object
   public String toString() {
     return TextFormat.shortDebugString(proto);

+ 20 - 30
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java

@@ -43,19 +43,19 @@ public class ReplaceDatanodeOnFailure {
 
     private final Condition condition;
 
-    private Policy(Condition condition) {
+    Policy(Condition condition) {
       this.condition = condition;
     }
-    
+
     Condition getCondition() {
       return condition;
     }
   }
 
   /** Datanode replacement condition */
-  private static interface Condition {
+  private interface Condition {
     /** Return true unconditionally. */
-    static final Condition TRUE = new Condition() {
+    Condition TRUE = new Condition() {
       @Override
       public boolean satisfy(short replication, DatanodeInfo[] existings,
           int nExistings, boolean isAppend, boolean isHflushed) {
@@ -64,7 +64,7 @@ public class ReplaceDatanodeOnFailure {
     };
 
     /** Return false unconditionally. */
-    static final Condition FALSE = new Condition() {
+    Condition FALSE = new Condition() {
       @Override
       public boolean satisfy(short replication, DatanodeInfo[] existings,
           int nExistings, boolean isAppend, boolean isHflushed) {
@@ -80,31 +80,24 @@ public class ReplaceDatanodeOnFailure {
      *   (1) floor(r/2) >= n; or
      *   (2) r > n and the block is hflushed/appended.
      */
-    static final Condition DEFAULT = new Condition() {
+    Condition DEFAULT = new Condition() {
       @Override
       public boolean satisfy(final short replication,
           final DatanodeInfo[] existings, final int n, final boolean isAppend,
           final boolean isHflushed) {
-        if (replication < 3) {
-          return false;
-        } else {
-          if (n <= (replication/2)) {
-            return true;
-          } else {
-            return isAppend || isHflushed;
-          }
-        }
+        return replication >= 3 &&
+            (n <= (replication / 2) || isAppend || isHflushed);
       }
     };
 
     /** Is the condition satisfied? */
-    public boolean satisfy(short replication, DatanodeInfo[] existings,
-        int nExistings, boolean isAppend, boolean isHflushed);
+    boolean satisfy(short replication, DatanodeInfo[] existings, int nExistings,
+                    boolean isAppend, boolean isHflushed);
   }
 
   private final Policy policy;
   private final boolean bestEffort;
-  
+
   public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
     this.policy = policy;
     this.bestEffort = bestEffort;
@@ -124,7 +117,7 @@ public class ReplaceDatanodeOnFailure {
    * Best effort means that the client will try to replace the failed datanode
    * (provided that the policy is satisfied), however, it will continue the
    * write operation in case that the datanode replacement also fails.
-   * 
+   *
    * @return Suppose the datanode replacement fails.
    *     false: An exception should be thrown so that the write will fail.
    *     true : The write should be resumed with the remaining datandoes.
@@ -137,16 +130,13 @@ public class ReplaceDatanodeOnFailure {
   public boolean satisfy(
       final short replication, final DatanodeInfo[] existings,
       final boolean isAppend, final boolean isHflushed) {
-    final int n = existings == null? 0: existings.length;
-    if (n == 0 || n >= replication) {
-      //don't need to add datanode for any policy.
-      return false;
-    } else {
-      return policy.getCondition().satisfy(
-          replication, existings, n, isAppend, isHflushed);
-    }
+    final int n = existings == null ? 0 : existings.length;
+    //don't need to add datanode for any policy.
+    return !(n == 0 || n >= replication) &&
+        policy.getCondition().satisfy(replication, existings, n, isAppend,
+            isHflushed);
   }
-  
+
   @Override
   public String toString() {
     return policy.toString();
@@ -158,7 +148,7 @@ public class ReplaceDatanodeOnFailure {
     final boolean bestEffort = conf.getBoolean(
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
-    
+
     return new ReplaceDatanodeOnFailure(policy, bestEffort);
   }
 
@@ -197,4 +187,4 @@ public class ReplaceDatanodeOnFailure {
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
         bestEffort);
   }
-}
+}

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

@@ -61,12 +61,11 @@ public class Sender implements DataTransferProtocol {
 
   /** Create a sender for DataTransferProtocol with a output stream. */
   public Sender(final DataOutputStream out) {
-    this.out = out;    
+    this.out = out;
   }
 
   /** Initialize a operation. */
-  private static void op(final DataOutput out, final Op op
-      ) throws IOException {
+  private static void op(final DataOutput out, final Op op) throws IOException {
     out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
     op.write(out);
   }
@@ -80,13 +79,14 @@ public class Sender implements DataTransferProtocol {
     out.flush();
   }
 
-  static private CachingStrategyProto getCachingStrategy(CachingStrategy cachingStrategy) {
+  static private CachingStrategyProto getCachingStrategy(
+      CachingStrategy cachingStrategy) {
     CachingStrategyProto.Builder builder = CachingStrategyProto.newBuilder();
     if (cachingStrategy.getReadahead() != null) {
-      builder.setReadahead(cachingStrategy.getReadahead().longValue());
+      builder.setReadahead(cachingStrategy.getReadahead());
     }
     if (cachingStrategy.getDropBehind() != null) {
-      builder.setDropBehind(cachingStrategy.getDropBehind().booleanValue());
+      builder.setDropBehind(cachingStrategy.getDropBehind());
     }
     return builder.build();
   }
@@ -101,24 +101,25 @@ public class Sender implements DataTransferProtocol {
       final CachingStrategy cachingStrategy) throws IOException {
 
     OpReadBlockProto proto = OpReadBlockProto.newBuilder()
-      .setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName, blockToken))
-      .setOffset(blockOffset)
-      .setLen(length)
-      .setSendChecksums(sendChecksum)
-      .setCachingStrategy(getCachingStrategy(cachingStrategy))
-      .build();
+        .setHeader(DataTransferProtoUtil.buildClientHeader(blk, clientName,
+            blockToken))
+        .setOffset(blockOffset)
+        .setLen(length)
+        .setSendChecksums(sendChecksum)
+        .setCachingStrategy(getCachingStrategy(cachingStrategy))
+        .build();
 
     send(out, Op.READ_BLOCK, proto);
   }
-  
+
 
   @Override
   public void writeBlock(final ExtendedBlock blk,
-      final StorageType storageType, 
+      final StorageType storageType,
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final DatanodeInfo[] targets,
-      final StorageType[] targetStorageTypes, 
+      final StorageType[] targetStorageTypes,
       final DatanodeInfo source,
       final BlockConstructionStage stage,
       final int pipelineSize,
@@ -132,26 +133,27 @@ public class Sender implements DataTransferProtocol {
       final boolean[] targetPinnings) throws IOException {
     ClientOperationHeaderProto header = DataTransferProtoUtil.buildClientHeader(
         blk, clientName, blockToken);
-    
+
     ChecksumProto checksumProto =
-      DataTransferProtoUtil.toProto(requestedChecksum);
+        DataTransferProtoUtil.toProto(requestedChecksum);
 
     OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
-      .setHeader(header)
-      .setStorageType(PBHelperClient.convertStorageType(storageType))
-      .addAllTargets(PBHelperClient.convert(targets, 1))
-      .addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes, 1))
-      .setStage(toProto(stage))
-      .setPipelineSize(pipelineSize)
-      .setMinBytesRcvd(minBytesRcvd)
-      .setMaxBytesRcvd(maxBytesRcvd)
-      .setLatestGenerationStamp(latestGenerationStamp)
-      .setRequestedChecksum(checksumProto)
-      .setCachingStrategy(getCachingStrategy(cachingStrategy))
-      .setAllowLazyPersist(allowLazyPersist)
-      .setPinning(pinning)
-      .addAllTargetPinnings(PBHelperClient.convert(targetPinnings, 1));
-    
+        .setHeader(header)
+        .setStorageType(PBHelperClient.convertStorageType(storageType))
+        .addAllTargets(PBHelperClient.convert(targets, 1))
+        .addAllTargetStorageTypes(
+            PBHelperClient.convertStorageTypes(targetStorageTypes, 1))
+        .setStage(toProto(stage))
+        .setPipelineSize(pipelineSize)
+        .setMinBytesRcvd(minBytesRcvd)
+        .setMaxBytesRcvd(maxBytesRcvd)
+        .setLatestGenerationStamp(latestGenerationStamp)
+        .setRequestedChecksum(checksumProto)
+        .setCachingStrategy(getCachingStrategy(cachingStrategy))
+        .setAllowLazyPersist(allowLazyPersist)
+        .setPinning(pinning)
+        .addAllTargetPinnings(PBHelperClient.convert(targetPinnings, 1));
+
     if (source != null) {
       proto.setSource(PBHelperClient.convertDatanodeInfo(source));
     }
@@ -165,13 +167,14 @@ public class Sender implements DataTransferProtocol {
       final String clientName,
       final DatanodeInfo[] targets,
       final StorageType[] targetStorageTypes) throws IOException {
-    
+
     OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
-      .setHeader(DataTransferProtoUtil.buildClientHeader(
-          blk, clientName, blockToken))
-      .addAllTargets(PBHelperClient.convert(targets))
-      .addAllTargetStorageTypes(PBHelperClient.convertStorageTypes(targetStorageTypes))
-      .build();
+        .setHeader(DataTransferProtoUtil.buildClientHeader(
+            blk, clientName, blockToken))
+        .addAllTargets(PBHelperClient.convert(targets))
+        .addAllTargetStorageTypes(
+            PBHelperClient.convertStorageTypes(targetStorageTypes))
+        .build();
 
     send(out, Op.TRANSFER_BLOCK, proto);
   }
@@ -180,11 +183,11 @@ public class Sender implements DataTransferProtocol {
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException {
+      throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
-          .setHeader(DataTransferProtoUtil.buildBaseHeader(
-            blk, blockToken)).setMaxVersion(maxVersion);
+            .setHeader(DataTransferProtoUtil.buildBaseHeader(
+                blk, blockToken)).setMaxVersion(maxVersion);
     if (slotId != null) {
       builder.setSlotId(PBHelperClient.convert(slotId));
     }
@@ -192,12 +195,12 @@ public class Sender implements DataTransferProtocol {
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }
-  
+
   @Override
   public void releaseShortCircuitFds(SlotId slotId) throws IOException {
     ReleaseShortCircuitAccessRequestProto.Builder builder =
         ReleaseShortCircuitAccessRequestProto.newBuilder().
-        setSlotId(PBHelperClient.convert(slotId));
+            setSlotId(PBHelperClient.convert(slotId));
     SpanId spanId = Tracer.getCurrentSpanId();
     if (spanId.isValid()) {
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
@@ -212,7 +215,7 @@ public class Sender implements DataTransferProtocol {
   public void requestShortCircuitShm(String clientName) throws IOException {
     ShortCircuitShmRequestProto.Builder builder =
         ShortCircuitShmRequestProto.newBuilder().
-        setClientName(clientName);
+            setClientName(clientName);
     SpanId spanId = Tracer.getCurrentSpanId();
     if (spanId.isValid()) {
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
@@ -222,20 +225,20 @@ public class Sender implements DataTransferProtocol {
     ShortCircuitShmRequestProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);
   }
-  
+
   @Override
   public void replaceBlock(final ExtendedBlock blk,
-      final StorageType storageType, 
+      final StorageType storageType,
       final Token<BlockTokenIdentifier> blockToken,
       final String delHint,
       final DatanodeInfo source) throws IOException {
     OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
-      .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
-      .setStorageType(PBHelperClient.convertStorageType(storageType))
-      .setDelHint(delHint)
-      .setSource(PBHelperClient.convertDatanodeInfo(source))
-      .build();
-    
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .setStorageType(PBHelperClient.convertStorageType(storageType))
+        .setDelHint(delHint)
+        .setSource(PBHelperClient.convertDatanodeInfo(source))
+        .build();
+
     send(out, Op.REPLACE_BLOCK, proto);
   }
 
@@ -243,9 +246,9 @@ public class Sender implements DataTransferProtocol {
   public void copyBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken) throws IOException {
     OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
-      .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
-      .build();
-    
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .build();
+
     send(out, Op.COPY_BLOCK, proto);
   }
 
@@ -253,9 +256,9 @@ public class Sender implements DataTransferProtocol {
   public void blockChecksum(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken) throws IOException {
     OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
-      .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
-      .build();
-    
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .build();
+
     send(out, Op.BLOCK_CHECKSUM, proto);
   }
 }

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

@@ -29,7 +29,7 @@ import org.apache.hadoop.util.ReflectionUtils;
  * The default implementation is to return false indicating that
  * the channel is not trusted.
  * This class can be overridden to provide custom logic to determine
- * whether a channel is trusted or not. 
+ * whether a channel is trusted or not.
  * The custom class can be specified via configuration.
  *
  */
@@ -39,14 +39,13 @@ public class TrustedChannelResolver implements Configurable {
   /**
    * Returns an instance of TrustedChannelResolver.
    * Looks up the configuration to see if there is custom class specified.
-   * @param conf
    * @return TrustedChannelResolver
    */
   public static TrustedChannelResolver getInstance(Configuration conf) {
     Class<? extends TrustedChannelResolver> clazz =
-      conf.getClass(
-          HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
-          TrustedChannelResolver.class, TrustedChannelResolver.class);
+        conf.getClass(
+            HdfsClientConfigKeys.DFS_TRUSTEDCHANNEL_RESOLVER_CLASS,
+            TrustedChannelResolver.class, TrustedChannelResolver.class);
     return ReflectionUtils.newInstance(clazz, conf);
   }
 

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

@@ -6,9 +6,9 @@
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -69,7 +69,7 @@ import com.google.protobuf.ByteString;
 public final class DataTransferSaslUtil {
 
   private static final Logger LOG = LoggerFactory.getLogger(
-    DataTransferSaslUtil.class);
+      DataTransferSaslUtil.class);
 
   /**
    * Delimiter for the three-part SASL username string.
@@ -97,20 +97,20 @@ public final class DataTransferSaslUtil {
       throw new IOException("Failed to complete SASL handshake");
     }
     Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
-      saslProps.get(Sasl.QOP).split(",")));
+        saslProps.get(Sasl.QOP).split(",")));
     String negotiatedQop = sasl.getNegotiatedQop();
     LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
-      requestedQop, negotiatedQop);
+        requestedQop, negotiatedQop);
     if (!requestedQop.contains(negotiatedQop)) {
       throw new IOException(String.format("SASL handshake completed, but " +
-        "channel does not have acceptable quality of protection, " +
-        "requested = %s, negotiated = %s", requestedQop, negotiatedQop));
+          "channel does not have acceptable quality of protection, " +
+          "requested = %s, negotiated = %s", requestedQop, negotiatedQop));
     }
   }
-  
+
   /**
    * Check whether requested SASL Qop contains privacy.
-   * 
+   *
    * @param saslProps properties of SASL negotiation
    * @return boolean true if privacy exists
    */
@@ -145,7 +145,7 @@ public final class DataTransferSaslUtil {
    */
   public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
     return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8)
-      .toCharArray();
+        .toCharArray();
   }
 
   /**
@@ -153,7 +153,6 @@ public final class DataTransferSaslUtil {
    * [host][/ip-address]:port.  The host may be missing.  The IP address (and
    * preceding '/') may be missing.  The port preceded by ':' is always present.
    *
-   * @param peer
    * @return InetAddress from peer
    */
   public static InetAddress getPeerAddress(Peer peer) {
@@ -181,23 +180,26 @@ public final class DataTransferSaslUtil {
     String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
     if (qops == null || qops.isEmpty()) {
       LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " +
-        "QOP found in configuration for {}", DFS_DATA_TRANSFER_PROTECTION_KEY);
+          "QOP found in configuration for {}",
+          DFS_DATA_TRANSFER_PROTECTION_KEY);
       return null;
     }
     Configuration saslPropsResolverConf = new Configuration(conf);
     saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
     Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
-      HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
-      SaslPropertiesResolver.class, SaslPropertiesResolver.class);
-    resolverClass = conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
-      resolverClass, SaslPropertiesResolver.class);
+        HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
+        SaslPropertiesResolver.class, SaslPropertiesResolver.class);
+    resolverClass =
+        conf.getClass(DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY,
+        resolverClass, SaslPropertiesResolver.class);
     saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
-      resolverClass, SaslPropertiesResolver.class);
+        resolverClass, SaslPropertiesResolver.class);
     SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
-      saslPropsResolverConf);
+        saslPropsResolverConf);
     LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
-      "QOP {} = {}, configured class {} = {}", DFS_DATA_TRANSFER_PROTECTION_KEY, qops, 
-      DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
+            "QOP {} = {}, configured class {} = {}",
+        DFS_DATA_TRANSFER_PROTECTION_KEY, qops,
+        DFS_DATA_TRANSFER_SASL_PROPS_RESOLVER_CLASS_KEY, resolverClass);
     return resolver;
   }
 
@@ -219,10 +221,10 @@ public final class DataTransferSaslUtil {
       return proto.getPayload().toByteArray();
     }
   }
-  
+
   /**
-   * Reads a SASL negotiation message and negotiation cipher options. 
-   * 
+   * Reads a SASL negotiation message and negotiation cipher options.
+   *
    * @param in stream to read
    * @param cipherOptions list to store negotiation cipher options
    * @return byte[] SASL negotiation message
@@ -246,10 +248,10 @@ public final class DataTransferSaslUtil {
       return proto.getPayload().toByteArray();
     }
   }
-  
+
   /**
    * Negotiate a cipher option which server supports.
-   * 
+   *
    * @param conf the configuration
    * @param options the cipher options which client supports
    * @return CipherOption negotiated cipher option
@@ -279,6 +281,7 @@ public final class DataTransferSaslUtil {
           byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
           byte[] outKey = new byte[keyLen];
           byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
+          assert codec != null;
           codec.generateSecureRandom(inKey);
           codec.generateSecureRandom(inIv);
           codec.generateSecureRandom(outKey);
@@ -289,21 +292,21 @@ public final class DataTransferSaslUtil {
     }
     return null;
   }
-  
+
   /**
    * Send SASL message and negotiated cipher option to client.
-   * 
+   *
    * @param out stream to receive message
    * @param payload to send
    * @param option negotiated cipher option
    * @throws IOException for any error
    */
   public static void sendSaslMessageAndNegotiatedCipherOption(
-      OutputStream out, byte[] payload, CipherOption option) 
-          throws IOException {
+      OutputStream out, byte[] payload, CipherOption option)
+      throws IOException {
     DataTransferEncryptorMessageProto.Builder builder =
         DataTransferEncryptorMessageProto.newBuilder();
-    
+
     builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
     if (payload != null) {
       builder.setPayload(ByteString.copyFrom(payload));
@@ -311,16 +314,16 @@ public final class DataTransferSaslUtil {
     if (option != null) {
       builder.addCipherOption(PBHelperClient.convert(option));
     }
-    
+
     DataTransferEncryptorMessageProto proto = builder.build();
     proto.writeDelimitedTo(out);
     out.flush();
   }
-  
+
   /**
    * Create IOStreamPair of {@link org.apache.hadoop.crypto.CryptoInputStream}
    * and {@link org.apache.hadoop.crypto.CryptoOutputStream}
-   * 
+   *
    * @param conf the configuration
    * @param cipherOption negotiated cipher option
    * @param out underlying output stream
@@ -330,7 +333,7 @@ public final class DataTransferSaslUtil {
    * @throws IOException for any error
    */
   public static IOStreamPair createStreamPair(Configuration conf,
-      CipherOption cipherOption, OutputStream out, InputStream in, 
+      CipherOption cipherOption, OutputStream out, InputStream in,
       boolean isServer) throws IOException {
     LOG.debug("Creating IOStreamPair of CryptoInputStream and "
         + "CryptoOutputStream.");
@@ -340,9 +343,9 @@ public final class DataTransferSaslUtil {
     byte[] inIv = cipherOption.getInIv();
     byte[] outKey = cipherOption.getOutKey();
     byte[] outIv = cipherOption.getOutIv();
-    InputStream cIn = new CryptoInputStream(in, codec, 
+    InputStream cIn = new CryptoInputStream(in, codec,
         isServer ? inKey : outKey, isServer ? inIv : outIv);
-    OutputStream cOut = new CryptoOutputStream(out, codec, 
+    OutputStream cOut = new CryptoOutputStream(out, codec,
         isServer ? outKey : inKey, isServer ? outIv : inIv);
     return new IOStreamPair(cIn, cOut);
   }
@@ -370,10 +373,10 @@ public final class DataTransferSaslUtil {
       throws IOException {
     sendSaslMessage(out, DataTransferEncryptorStatus.SUCCESS, payload, null);
   }
-  
+
   /**
    * Send a SASL negotiation message and negotiation cipher options to server.
-   * 
+   *
    * @param out stream to receive message
    * @param payload to send
    * @param options cipher options to negotiate
@@ -381,10 +384,10 @@ public final class DataTransferSaslUtil {
    */
   public static void sendSaslMessageAndNegotiationCipherOptions(
       OutputStream out, byte[] payload, List<CipherOption> options)
-          throws IOException {
+      throws IOException {
     DataTransferEncryptorMessageProto.Builder builder =
         DataTransferEncryptorMessageProto.newBuilder();
-    
+
     builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
     if (payload != null) {
       builder.setPayload(ByteString.copyFrom(payload));
@@ -392,23 +395,23 @@ public final class DataTransferSaslUtil {
     if (options != null) {
       builder.addAllCipherOption(PBHelperClient.convertCipherOptions(options));
     }
-    
+
     DataTransferEncryptorMessageProto proto = builder.build();
     proto.writeDelimitedTo(out);
     out.flush();
   }
-  
+
   /**
    * Read SASL message and negotiated cipher option from server.
-   * 
+   *
    * @param in stream to read
-   * @return SaslResponseWithNegotiatedCipherOption SASL message and 
+   * @return SaslResponseWithNegotiatedCipherOption SASL message and
    * negotiated cipher option
    * @throws IOException for any error
    */
   public static SaslResponseWithNegotiatedCipherOption
       readSaslMessageAndNegotiatedCipherOption(InputStream in)
-          throws IOException {
+      throws IOException {
     DataTransferEncryptorMessageProto proto =
         DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
     if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
@@ -426,17 +429,17 @@ public final class DataTransferSaslUtil {
       return new SaslResponseWithNegotiatedCipherOption(response, option);
     }
   }
-  
+
   /**
    * Encrypt the key and iv of the negotiated cipher option.
-   * 
+   *
    * @param option negotiated cipher option
    * @param sasl SASL participant representing server
-   * @return CipherOption negotiated cipher option which contains the 
+   * @return CipherOption negotiated cipher option which contains the
    * encrypted key and iv
    * @throws IOException for any error
    */
-  public static CipherOption wrap(CipherOption option, SaslParticipant sasl) 
+  public static CipherOption wrap(CipherOption option, SaslParticipant sasl)
       throws IOException {
     if (option != null) {
       byte[] inKey = option.getInKey();
@@ -450,16 +453,16 @@ public final class DataTransferSaslUtil {
       return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
           outKey, option.getOutIv());
     }
-    
+
     return null;
   }
-  
+
   /**
    * Decrypt the key and iv of the negotiated cipher option.
-   * 
+   *
    * @param option negotiated cipher option
    * @param sasl SASL participant representing client
-   * @return CipherOption negotiated cipher option which contains the 
+   * @return CipherOption negotiated cipher option which contains the
    * decrypted key and iv
    * @throws IOException for any error
    */
@@ -477,7 +480,7 @@ public final class DataTransferSaslUtil {
       return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
           outKey, option.getOutIv());
     }
-    
+
     return null;
   }
 
@@ -492,10 +495,10 @@ public final class DataTransferSaslUtil {
    */
   public static void sendSaslMessage(OutputStream out,
       DataTransferEncryptorStatus status, byte[] payload, String message)
-          throws IOException {
+      throws IOException {
     DataTransferEncryptorMessageProto.Builder builder =
         DataTransferEncryptorMessageProto.newBuilder();
-    
+
     builder.setStatus(status);
     if (payload != null) {
       builder.setPayload(ByteString.copyFrom(payload));
@@ -503,7 +506,7 @@ public final class DataTransferSaslUtil {
     if (message != null) {
       builder.setMessage(message);
     }
-    
+
     DataTransferEncryptorMessageProto proto = builder.build();
     proto.writeDelimitedTo(out);
     out.flush();

+ 52 - 56
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java

@@ -76,7 +76,7 @@ import com.google.common.collect.Lists;
 public class SaslDataTransferClient {
 
   private static final Logger LOG = LoggerFactory.getLogger(
-    SaslDataTransferClient.class);
+      SaslDataTransferClient.class);
 
   private final Configuration conf;
   private final AtomicBoolean fallbackToSimpleAuth;
@@ -94,7 +94,7 @@ public class SaslDataTransferClient {
    * @param trustedChannelResolver for identifying trusted connections that do
    *   not require SASL negotiation
    */
-  public SaslDataTransferClient(Configuration conf, 
+  public SaslDataTransferClient(Configuration conf,
       SaslPropertiesResolver saslPropsResolver,
       TrustedChannelResolver trustedChannelResolver) {
     this(conf, saslPropsResolver, trustedChannelResolver, null);
@@ -110,7 +110,7 @@ public class SaslDataTransferClient {
    * @param fallbackToSimpleAuth checked on each attempt at general SASL
    *   handshake, if true forces use of simple auth
    */
-  public SaslDataTransferClient(Configuration conf, 
+  public SaslDataTransferClient(Configuration conf,
       SaslPropertiesResolver saslPropsResolver,
       TrustedChannelResolver trustedChannelResolver,
       AtomicBoolean fallbackToSimpleAuth) {
@@ -138,9 +138,9 @@ public class SaslDataTransferClient {
       throws IOException {
     // The encryption key factory only returns a key if encryption is enabled.
     DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
-      encryptionKeyFactory.newDataEncryptionKey() : null;
+        encryptionKeyFactory.newDataEncryptionKey() : null;
     IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
-      underlyingIn, encryptionKey, accessToken, datanodeId);
+        underlyingIn, encryptionKey, accessToken, datanodeId);
     return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
   }
 
@@ -158,8 +158,8 @@ public class SaslDataTransferClient {
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       throws IOException {
     IOStreamPair ios = checkTrustAndSend(getPeerAddress(peer),
-      peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
-      accessToken, datanodeId);
+        peer.getOutputStream(), peer.getInputStream(), encryptionKeyFactory,
+        accessToken, datanodeId);
     // TODO: Consider renaming EncryptedPeer to SaslPeer.
     return ios != null ? new EncryptedPeer(peer, ios) : peer;
   }
@@ -181,7 +181,7 @@ public class SaslDataTransferClient {
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       throws IOException {
     IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
-      underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
+        underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
     return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
   }
 
@@ -207,13 +207,13 @@ public class SaslDataTransferClient {
         !trustedChannelResolver.isTrusted(addr)) {
       // The encryption key factory only returns a key if encryption is enabled.
       DataEncryptionKey encryptionKey =
-        encryptionKeyFactory.newDataEncryptionKey();
+          encryptionKeyFactory.newDataEncryptionKey();
       return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
-        datanodeId);
+          datanodeId);
     } else {
       LOG.debug(
-        "SASL client skipping handshake on trusted connection for addr = {}, "
-        + "datanodeId = {}", addr, datanodeId);
+          "SASL client skipping handshake on trusted connection for addr = {}, "
+              + "datanodeId = {}", addr, datanodeId);
       return null;
     }
   }
@@ -236,40 +236,38 @@ public class SaslDataTransferClient {
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       throws IOException {
     if (encryptionKey != null) {
-      LOG.debug(
-        "SASL client doing encrypted handshake for addr = {}, datanodeId = {}",
-        addr, datanodeId);
+      LOG.debug("SASL client doing encrypted handshake for addr = {}, "
+          + "datanodeId = {}", addr, datanodeId);
       return getEncryptedStreams(underlyingOut, underlyingIn,
-        encryptionKey);
+          encryptionKey);
     } else if (!UserGroupInformation.isSecurityEnabled()) {
-      LOG.debug(
-        "SASL client skipping handshake in unsecured configuration for "
-        + "addr = {}, datanodeId = {}", addr, datanodeId);
+      LOG.debug("SASL client skipping handshake in unsecured configuration for "
+          + "addr = {}, datanodeId = {}", addr, datanodeId);
       return null;
     } else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
       LOG.debug(
-        "SASL client skipping handshake in secured configuration with "
-        + "privileged port for addr = {}, datanodeId = {}", addr, datanodeId);
+          "SASL client skipping handshake in secured configuration with "
+              + "privileged port for addr = {}, datanodeId = {}",
+          addr, datanodeId);
       return null;
     } else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
       LOG.debug(
-        "SASL client skipping handshake in secured configuration with "
-        + "unsecured cluster for addr = {}, datanodeId = {}", addr, datanodeId);
+          "SASL client skipping handshake in secured configuration with "
+              + "unsecured cluster for addr = {}, datanodeId = {}",
+          addr, datanodeId);
       return null;
     } else if (saslPropsResolver != null) {
       LOG.debug(
-        "SASL client doing general handshake for addr = {}, datanodeId = {}",
-        addr, datanodeId);
-      return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken,
-        datanodeId);
+          "SASL client doing general handshake for addr = {}, datanodeId = {}",
+          addr, datanodeId);
+      return getSaslStreams(addr, underlyingOut, underlyingIn, accessToken);
     } else {
       // It's a secured cluster using non-privileged ports, but no SASL.  The
       // only way this can happen is if the DataNode has
-      // ignore.secure.ports.for.testing configured, so this is a rare edge case.
-      LOG.debug(
-        "SASL client skipping handshake in secured configuration with no SASL "
-        + "protection configured for addr = {}, datanodeId = {}",
-        addr, datanodeId);
+      // ignore.secure.ports.for.testing configured so this is a rare edge case.
+      LOG.debug("SASL client skipping handshake in secured configuration with "
+              + "no SASL protection configured for addr = {}, datanodeId = {}",
+          addr, datanodeId);
       return null;
     }
   }
@@ -287,24 +285,24 @@ public class SaslDataTransferClient {
       InputStream underlyingIn, DataEncryptionKey encryptionKey)
       throws IOException {
     Map<String, String> saslProps = createSaslPropertiesForEncryption(
-      encryptionKey.encryptionAlgorithm);
+        encryptionKey.encryptionAlgorithm);
 
     LOG.debug("Client using encryption algorithm {}",
-      encryptionKey.encryptionAlgorithm);
+        encryptionKey.encryptionAlgorithm);
 
     String userName = getUserNameFromEncryptionKey(encryptionKey);
     char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
     CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
-      password);
+        password);
     return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
-      callbackHandler);
+        callbackHandler);
   }
 
   /**
    * The SASL username for an encrypted handshake consists of the keyId,
    * blockPoolId, and nonce with the first two encoded as Strings, and the third
    * encoded using Base64. The fields are each separated by a single space.
-   * 
+   *
    * @param encryptionKey the encryption key to encode as a SASL username.
    * @return encoded username containing keyId, blockPoolId, and nonce
    */
@@ -312,7 +310,8 @@ public class SaslDataTransferClient {
       DataEncryptionKey encryptionKey) {
     return encryptionKey.keyId + NAME_DELIMITER +
         encryptionKey.blockPoolId + NAME_DELIMITER +
-        new String(Base64.encodeBase64(encryptionKey.nonce, false), Charsets.UTF_8);
+        new String(Base64.encodeBase64(encryptionKey.nonce, false),
+            Charsets.UTF_8);
   }
 
   /**
@@ -328,7 +327,7 @@ public class SaslDataTransferClient {
      * Creates a new SaslClientCallbackHandler.
      *
      * @param userName SASL user name
-     * @Param password SASL password
+     * @param password SASL password
      */
     public SaslClientCallbackHandler(String userName, char[] password) {
       this.password = password;
@@ -342,15 +341,13 @@ public class SaslDataTransferClient {
       PasswordCallback pc = null;
       RealmCallback rc = null;
       for (Callback callback : callbacks) {
-        if (callback instanceof RealmChoiceCallback) {
-          continue;
-        } else if (callback instanceof NameCallback) {
+        if (callback instanceof NameCallback) {
           nc = (NameCallback) callback;
         } else if (callback instanceof PasswordCallback) {
           pc = (PasswordCallback) callback;
         } else if (callback instanceof RealmCallback) {
           rc = (RealmCallback) callback;
-        } else {
+        } else if (!(callback instanceof RealmChoiceCallback)) {
           throw new UnsupportedCallbackException(callback,
               "Unrecognized SASL client callback");
         }
@@ -374,22 +371,21 @@ public class SaslDataTransferClient {
    * @param underlyingOut connection output stream
    * @param underlyingIn connection input stream
    * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
    * @return new pair of streams, wrapped after SASL negotiation
    * @throws IOException for any error
    */
   private IOStreamPair getSaslStreams(InetAddress addr,
       OutputStream underlyingOut, InputStream underlyingIn,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      Token<BlockTokenIdentifier> accessToken)
       throws IOException {
     Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
 
     String userName = buildUserName(accessToken);
     char[] password = buildClientPassword(accessToken);
     CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
-      password);
+        password);
     return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
-      callbackHandler);
+        callbackHandler);
   }
 
   /**
@@ -404,7 +400,7 @@ public class SaslDataTransferClient {
    */
   private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
     return new String(Base64.encodeBase64(blockToken.getIdentifier(), false),
-      Charsets.UTF_8);
+        Charsets.UTF_8);
   }
 
   /**
@@ -413,10 +409,10 @@ public class SaslDataTransferClient {
    *
    * @param blockToken for block access
    * @return SASL password
-   */    
+   */
   private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
     return new String(Base64.encodeBase64(blockToken.getPassword(), false),
-      Charsets.UTF_8).toCharArray();
+        Charsets.UTF_8).toCharArray();
   }
 
   /**
@@ -438,7 +434,7 @@ public class SaslDataTransferClient {
     DataInputStream in = new DataInputStream(underlyingIn);
 
     SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
-      saslProps, callbackHandler);
+        saslProps, callbackHandler);
 
     out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
     out.flush();
@@ -467,11 +463,11 @@ public class SaslDataTransferClient {
           cipherOptions.add(option);
         }
       }
-      sendSaslMessageAndNegotiationCipherOptions(out, localResponse, 
+      sendSaslMessageAndNegotiationCipherOptions(out, localResponse,
           cipherOptions);
 
       // step 2 (client-side only)
-      SaslResponseWithNegotiatedCipherOption response = 
+      SaslResponseWithNegotiatedCipherOption response =
           readSaslMessageAndNegotiatedCipherOption(in);
       localResponse = sasl.evaluateChallengeOrResponse(response.payload);
       assert localResponse == null;
@@ -485,11 +481,11 @@ public class SaslDataTransferClient {
         cipherOption = unwrap(response.cipherOption, sasl);
       }
 
-      // If negotiated cipher option is not null, we will use it to create 
+      // If negotiated cipher option is not null, we will use it to create
       // stream pair.
       return cipherOption != null ? createStreamPair(
-          conf, cipherOption, underlyingOut, underlyingIn, false) : 
-            sasl.createStreamPair(out, in);
+          conf, cipherOption, underlyingOut, underlyingIn, false) :
+          sasl.createStreamPair(out, in);
     } catch (IOException ioe) {
       sendGenericSaslErrorMessage(out, ioe.getMessage());
       throw ioe;

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java

@@ -129,20 +129,20 @@ class SaslParticipant {
       return (String) saslServer.getNegotiatedProperty(Sasl.QOP);
     }
   }
-  
+
   /**
    * After successful SASL negotiation, returns whether it's QOP privacy
-   * 
+   *
    * @return boolean whether it's QOP privacy
    */
   public boolean isNegotiatedQopPrivacy() {
     String qop = getNegotiatedQop();
     return qop != null && "auth-conf".equalsIgnoreCase(qop);
   }
-  
+
   /**
    * Wraps a byte array.
-   * 
+   *
    * @param bytes The array containing the bytes to wrap.
    * @param off The starting position at the array
    * @param len The number of bytes to wrap
@@ -156,10 +156,10 @@ class SaslParticipant {
       return saslServer.wrap(bytes, off, len);
     }
   }
-  
+
   /**
    * Unwraps a byte array.
-   * 
+   *
    * @param bytes The array containing the bytes to unwrap.
    * @param off The starting position at the array
    * @param len The number of bytes to unwrap

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

@@ -24,10 +24,10 @@ import org.apache.hadoop.crypto.CipherOption;
 public class SaslResponseWithNegotiatedCipherOption {
   final byte[] payload;
   final CipherOption cipherOption;
-  
-  public SaslResponseWithNegotiatedCipherOption(byte[] payload, 
+
+  public SaslResponseWithNegotiatedCipherOption(byte[] payload,
       CipherOption cipherOption) {
     this.payload = payload;
     this.cipherOption = cipherOption;
   }
-}
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java

@@ -28,7 +28,7 @@ import org.apache.hadoop.security.token.TokenInfo;
 @KerberosInfo(
     serverPrincipal = HdfsClientConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY)
 @TokenInfo(BlockTokenSelector.class)
-@ProtocolInfo(protocolName = 
+@ProtocolInfo(protocolName =
     "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol",
     protocolVersion = 1)
 @InterfaceAudience.Private

+ 20 - 16
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java

@@ -92,11 +92,11 @@ public class ClientDatanodeProtocolTranslatorPB implements
     ProtocolTranslator, Closeable {
   public static final Logger LOG = LoggerFactory
       .getLogger(ClientDatanodeProtocolTranslatorPB.class);
-  
+
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final ClientDatanodeProtocolPB rpcProxy;
-  private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES = 
+  private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES =
       RefreshNamenodesRequestProto.newBuilder().build();
   private final static GetDatanodeInfoRequestProto VOID_GET_DATANODE_INFO =
       GetDatanodeInfoRequestProto.newBuilder().build();
@@ -114,16 +114,16 @@ public class ClientDatanodeProtocolTranslatorPB implements
   public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
       Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
       LocatedBlock locatedBlock) throws IOException {
-    rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf, 
-                  socketTimeout, connectToDnViaHostname, locatedBlock);
+    rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf,
+        socketTimeout, connectToDnViaHostname, locatedBlock);
   }
-  
+
   public ClientDatanodeProtocolTranslatorPB(InetSocketAddress addr,
       UserGroupInformation ticket, Configuration conf, SocketFactory factory)
       throws IOException {
     rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory, 0);
   }
-  
+
   /**
    * Constructor.
    * @param datanodeid Datanode to connect to.
@@ -145,7 +145,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
 
   static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
       DatanodeID datanodeid, Configuration conf, int socketTimeout,
-      boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
+      boolean connectToDnViaHostname, LocatedBlock locatedBlock)
+      throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
     LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
@@ -167,7 +168,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
     return createClientDatanodeProtocolProxy(addr, ticket, confWithNoIpcIdle,
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
   }
-  
+
   static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
       InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
       SocketFactory factory, int socketTimeout) throws IOException {
@@ -185,8 +186,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
 
   @Override
   public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
-    GetReplicaVisibleLengthRequestProto req = GetReplicaVisibleLengthRequestProto
-        .newBuilder().setBlock(PBHelperClient.convert(b)).build();
+    GetReplicaVisibleLengthRequestProto req =
+        GetReplicaVisibleLengthRequestProto.newBuilder()
+            .setBlock(PBHelperClient.convert(b)).build();
     try {
       return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
     } catch (ServiceException e) {
@@ -219,8 +221,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
       Token<BlockTokenIdentifier> token) throws IOException {
     GetBlockLocalPathInfoRequestProto req =
         GetBlockLocalPathInfoRequestProto.newBuilder()
-        .setBlock(PBHelperClient.convert(block))
-        .setToken(PBHelperClient.convert(token)).build();
+            .setBlock(PBHelperClient.convert(block))
+            .setToken(PBHelperClient.convert(token)).build();
     GetBlockLocalPathInfoResponseProto resp;
     try {
       resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
@@ -294,7 +296,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
   public DatanodeLocalInfo getDatanodeInfo() throws IOException {
     GetDatanodeInfoResponseProto response;
     try {
-      response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER, VOID_GET_DATANODE_INFO);
+      response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER,
+          VOID_GET_DATANODE_INFO);
       return PBHelperClient.convert(response.getLocalInfo());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -311,7 +314,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
   }
 
   @Override
-  public ReconfigurationTaskStatus getReconfigurationStatus() throws IOException {
+  public ReconfigurationTaskStatus getReconfigurationStatus()
+      throws IOException {
     GetReconfigurationStatusResponseProto response;
     Map<PropertyChange, Optional<String>> statusMap = null;
     long startTime;
@@ -361,8 +365,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
     try {
       rpcProxy.triggerBlockReport(NULL_CONTROLLER,
           TriggerBlockReportRequestProto.newBuilder().
-            setIncremental(options.isIncremental()).
-            build());
+              setIncremental(options.isIncremental()).
+              build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }

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

@@ -37,10 +37,10 @@ import org.apache.hadoop.security.token.TokenInfo;
     protocolVersion = 1)
 /**
  * Protocol that a clients use to communicate with the NameNode.
- * 
+ *
  * Note: This extends the protocolbuffer service based interface to
  * add annotations required for security.
  */
-public interface ClientNamenodeProtocolPB extends 
-  ClientNamenodeProtocol.BlockingInterface {
+public interface ClientNamenodeProtocolPB extends
+    ClientNamenodeProtocol.BlockingInterface {
 }

+ 137 - 184
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.protocolPB;
 
 import java.io.Closeable;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.EnumSet;
@@ -33,12 +32,9 @@ import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -46,7 +42,6 @@ import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -54,7 +49,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
@@ -67,7 +61,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -171,8 +164,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
@@ -181,7 +172,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
@@ -206,37 +196,37 @@ public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
 
-  static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST = 
-  GetServerDefaultsRequestProto.newBuilder().build();
+  static final GetServerDefaultsRequestProto VOID_GET_SERVER_DEFAULT_REQUEST =
+      GetServerDefaultsRequestProto.newBuilder().build();
 
   private final static GetFsStatusRequestProto VOID_GET_FSSTATUS_REQUEST =
-  GetFsStatusRequestProto.newBuilder().build();
+      GetFsStatusRequestProto.newBuilder().build();
 
   private final static SaveNamespaceRequestProto VOID_SAVE_NAMESPACE_REQUEST =
   SaveNamespaceRequestProto.newBuilder().build();
 
-  private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST = 
+  private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST =
   RollEditsRequestProto.getDefaultInstance();
 
   private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
-  RefreshNodesRequestProto.newBuilder().build();
+      RefreshNodesRequestProto.newBuilder().build();
 
   private final static FinalizeUpgradeRequestProto
-  VOID_FINALIZE_UPGRADE_REQUEST =
+      VOID_FINALIZE_UPGRADE_REQUEST =
       FinalizeUpgradeRequestProto.newBuilder().build();
 
   private final static GetDataEncryptionKeyRequestProto
-  VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
+      VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
       GetDataEncryptionKeyRequestProto.newBuilder().build();
 
   private final static GetStoragePoliciesRequestProto
-  VOID_GET_STORAGE_POLICIES_REQUEST =
+      VOID_GET_STORAGE_POLICIES_REQUEST =
       GetStoragePoliciesRequestProto.newBuilder().build();
 
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
   }
-  
+
   @Override
   public void close() {
     RPC.stopProxy(rpcProxy);
@@ -244,8 +234,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public LocatedBlocks getBlockLocations(String src, long offset, long length)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
         .newBuilder()
         .setSrc(src)
@@ -256,7 +245,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
           req);
       return resp.hasLocations() ?
-        PBHelperClient.convert(resp.getLocations()) : null;
+          PBHelperClient.convert(resp.getLocations()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -276,13 +265,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public HdfsFileStatus create(String src, FsPermission masked,
       String clientName, EnumSetWritable<CreateFlag> flag,
-      boolean createParent, short replication, long blockSize, 
+      boolean createParent, short replication, long blockSize,
       CryptoProtocolVersion[] supportedVersions)
-      throws AccessControlException, AlreadyBeingCreatedException,
-      DSQuotaExceededException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      throws IOException {
     CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
         .setSrc(src)
         .setMasked(PBHelperClient.convert(masked))
@@ -291,7 +276,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setCreateParent(createParent)
         .setReplication(replication)
         .setBlockSize(blockSize);
-    builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
+    builder.addAllCryptoProtocolVersion(
+        PBHelperClient.convert(supportedVersions));
     CreateRequestProto req = builder.build();
     try {
       CreateResponseProto res = rpcProxy.create(null, req);
@@ -304,7 +290,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public boolean truncate(String src, long newLength, String clientName)
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     TruncateRequestProto req = TruncateRequestProto.newBuilder()
         .setSrc(src)
         .setNewLength(newLength)
@@ -319,18 +305,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public LastBlockWithStatus append(String src, String clientName,
-      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
-      DSQuotaExceededException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      EnumSetWritable<CreateFlag> flag) throws IOException {
     AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
-        .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
+        .setClientName(clientName).setFlag(
+            PBHelperClient.convertCreateFlag(flag))
         .build();
     try {
       AppendResponseProto res = rpcProxy.append(null, req);
       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
           .convert(res.getBlock()) : null;
-      HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat())
-          : null;
+      HdfsFileStatus stat = (res.hasStat()) ?
+          PBHelperClient.convert(res.getStat()) : null;
       return new LastBlockWithStatus(lastBlock, stat);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -339,9 +324,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      throws IOException {
     SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
         .setSrc(src)
         .setReplication(replication)
@@ -355,8 +338,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
         .setSrc(src)
         .setPermission(PBHelperClient.convert(permission))
@@ -370,14 +352,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
         .setSrc(src);
     if (username != null)
-        req.setUsername(username);
+      req.setUsername(username);
     if (groupname != null)
-        req.setGroupname(groupname);
+      req.setGroupname(groupname);
     try {
       rpcProxy.setOwner(null, req.build());
     } catch (ServiceException e) {
@@ -387,28 +368,24 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void abandonBlock(ExtendedBlock b, long fileId, String src,
-      String holder) throws AccessControlException, FileNotFoundException,
-        UnresolvedLinkException, IOException {
+      String holder) throws IOException {
     AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
         .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
-            .setFileId(fileId).build();
+        .setFileId(fileId).build();
     try {
       rpcProxy.abandonBlock(null, req);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
   @Override
   public LocatedBlock addBlock(String src, String clientName,
       ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
-      String[] favoredNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      String[] favoredNodes) throws IOException {
     AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
         .setSrc(src).setClientName(clientName).setFileId(fileId);
-    if (previous != null) 
+    if (previous != null)
       req.setPrevious(PBHelperClient.convert(previous));
     if (excludeNodes != null)
       req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
@@ -425,10 +402,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public LocatedBlock getAdditionalDatanode(String src, long fileId,
       ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
-      DatanodeInfo[] excludes,
-      int numAdditionalNodes, String clientName) throws AccessControlException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      DatanodeInfo[] excludes, int numAdditionalNodes, String clientName)
+      throws IOException {
     GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
         .newBuilder()
         .setSrc(src)
@@ -450,9 +425,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public boolean complete(String src, String clientName,
-                          ExtendedBlock last, long fileId)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      ExtendedBlock last, long fileId) throws IOException {
     CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
         .setSrc(src)
         .setClientName(clientName)
@@ -469,7 +442,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks)))
+        .addAllBlocks(Arrays.asList(
+            PBHelperClient.convertLocatedBlock(blocks)))
         .build();
     try {
       rpcProxy.reportBadBlocks(null, req);
@@ -479,8 +453,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public boolean rename(String src, String dst) throws UnresolvedLinkException,
-      IOException {
+  public boolean rename(String src, String dst) throws IOException {
     RenameRequestProto req = RenameRequestProto.newBuilder()
         .setSrc(src)
         .setDst(dst).build();
@@ -490,14 +463,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
 
   @Override
   public void rename2(String src, String dst, Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     boolean overwrite = false;
     if (options != null) {
       for (Rename option : options) {
@@ -519,8 +489,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void concat(String trg, String[] srcs) throws IOException,
-      UnresolvedLinkException {
+  public void concat(String trg, String[] srcs) throws IOException {
     ConcatRequestProto req = ConcatRequestProto.newBuilder().
         setTrg(trg).
         addAllSrcs(Arrays.asList(srcs)).build();
@@ -533,10 +502,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
-    DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src).setRecursive(recursive).build();
+  public boolean delete(String src, boolean recursive) throws IOException {
+    DeleteRequestProto req = DeleteRequestProto.newBuilder().setSrc(src)
+        .setRecursive(recursive).build();
     try {
       return rpcProxy.delete(null, req).getResult();
     } catch (ServiceException e) {
@@ -546,10 +514,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      throws IOException {
     MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
         .setSrc(src)
         .setMasked(PBHelperClient.convert(masked))
@@ -564,15 +529,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
+      boolean needLocation) throws IOException {
     GetListingRequestProto req = GetListingRequestProto.newBuilder()
         .setSrc(src)
         .setStartAfter(ByteString.copyFrom(startAfter))
         .setNeedLocation(needLocation).build();
     try {
       GetListingResponseProto result = rpcProxy.getListing(null, req);
-      
+
       if (result.hasDirList()) {
         return PBHelperClient.convert(result.getDirList());
       }
@@ -583,8 +547,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException {
+  public void renewLease(String clientName) throws IOException {
     RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
         .setClientName(clientName).build();
     try {
@@ -604,7 +567,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       return rpcProxy.recoverLease(null, req).getResult();
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
-    }  
+    }
   }
 
   @Override
@@ -632,22 +595,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
-      throws IOException {
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      DatanodeReportType type) throws IOException {
     final GetDatanodeStorageReportRequestProto req
         = GetDatanodeStorageReportRequestProto.newBuilder()
-            .setType(PBHelperClient.convert(type)).build();
+        .setType(PBHelperClient.convert(type)).build();
     try {
       return PBHelperClient.convertDatanodeStorageReports(
-          rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
+          rpcProxy.getDatanodeStorageReport(null, req)
+              .getDatanodeStorageReportsList());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
   @Override
-  public long getPreferredBlockSize(String filename) throws IOException,
-      UnresolvedLinkException {
+  public long getPreferredBlockSize(String filename) throws IOException {
     GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
         .newBuilder()
         .setFilename(filename)
@@ -660,9 +623,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
+  public boolean setSafeMode(SafeModeAction action, boolean isChecked)
+      throws IOException {
     SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
-        .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
+        .setAction(PBHelperClient.convert(action))
+        .setChecked(isChecked).build();
     try {
       return rpcProxy.setSafeMode(null, req).getResult();
     } catch (ServiceException e) {
@@ -671,16 +636,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void saveNamespace() throws AccessControlException, IOException {
+  public void saveNamespace() throws IOException {
     try {
       rpcProxy.saveNamespace(null, VOID_SAVE_NAMESPACE_REQUEST);
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
   @Override
-  public long rollEdits() throws AccessControlException, IOException {
+  public long rollEdits() throws IOException {
     try {
       RollEditsResponseProto resp = rpcProxy.rollEdits(null,
           VOID_ROLLEDITS_REQUEST);
@@ -691,8 +656,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException{
+  public boolean restoreFailedStorage(String arg) throws IOException{
     RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
         .newBuilder()
         .setArg(arg).build();
@@ -722,11 +686,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+      throws IOException {
     final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
         .setAction(PBHelperClient.convert(action)).build();
     try {
-      final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
+      final RollingUpgradeResponseProto proto =
+          rpcProxy.rollingUpgrade(null, r);
       if (proto.hasRollingUpgradeInfo()) {
         return PBHelperClient.convert(proto.getRollingUpgradeInfo());
       }
@@ -739,9 +705,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
-    ListCorruptFileBlocksRequestProto.Builder req = 
-        ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);   
-    if (cookie != null) 
+    ListCorruptFileBlocksRequestProto.Builder req =
+        ListCorruptFileBlocksRequestProto.newBuilder().setPath(path);
+    if (cookie != null)
       req.setCookie(cookie);
     try {
       return PBHelperClient.convert(
@@ -764,8 +730,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
+  public HdfsFileStatus getFileInfo(String src) throws IOException {
     GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
         .setSrc(src).build();
     try {
@@ -777,23 +742,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException {
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
     GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
         .setSrc(src).build();
     try {
       GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
-      return result.hasFs() ?  
-          PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) : null;
+      return result.hasFs() ?
+          PBHelperClient.convert(rpcProxy.getFileLinkInfo(null, req).getFs()) :
+          null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
   @Override
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+  public ContentSummary getContentSummary(String path) throws IOException {
     GetContentSummaryRequestProto req = GetContentSummaryRequestProto
         .newBuilder()
         .setPath(path)
@@ -808,9 +771,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
-                       StorageType type)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+      StorageType type) throws IOException {
     final SetQuotaRequestProto.Builder builder
         = SetQuotaRequestProto.newBuilder()
         .setPath(path)
@@ -829,12 +790,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void fsync(String src, long fileId, String client,
-                    long lastBlockLength)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+      long lastBlockLength) throws IOException {
     FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
         .setClient(client).setLastBlockLength(lastBlockLength)
-            .setFileId(fileId).build();
+        .setFileId(fileId).build();
     try {
       rpcProxy.fsync(null, req);
     } catch (ServiceException e) {
@@ -843,9 +802,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+  public void setTimes(String src, long mtime, long atime) throws IOException {
     SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
         .setSrc(src)
         .setMtime(mtime)
@@ -860,10 +817,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void createSymlink(String target, String link, FsPermission dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      boolean createParent) throws IOException {
     CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
         .setTarget(target)
         .setLink(link)
@@ -878,8 +832,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException {
+  public String getLinkTarget(String path) throws IOException {
     GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
         .setPath(path).build();
     try {
@@ -908,7 +861,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
-      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs)
+      throws IOException {
     UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
         .setClientName(clientName)
         .setOldBlock(PBHelperClient.convert(oldBlock))
@@ -931,9 +885,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setRenewer(renewer == null ? "" : renewer.toString())
         .build();
     try {
-      GetDelegationTokenResponseProto resp = rpcProxy.getDelegationToken(null, req);
-      return resp.hasToken() ? PBHelperClient.convertDelegationToken(resp.getToken())
-          : null;
+      GetDelegationTokenResponseProto resp =
+          rpcProxy.getDelegationToken(null, req);
+      return resp.hasToken() ?
+          PBHelperClient.convertDelegationToken(resp.getToken()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -942,9 +897,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
-    RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
-        setToken(PBHelperClient.convert(token)).
-        build();
+    RenewDelegationTokenRequestProto req =
+        RenewDelegationTokenRequestProto.newBuilder().
+            setToken(PBHelperClient.convert(token)).
+            build();
     try {
       return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
     } catch (ServiceException e) {
@@ -968,9 +924,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void setBalancerBandwidth(long bandwidth) throws IOException {
-    SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
-        .setBandwidth(bandwidth)
-        .build();
+    SetBalancerBandwidthRequestProto req =
+        SetBalancerBandwidthRequestProto.newBuilder()
+            .setBandwidth(bandwidth)
+            .build();
     try {
       rpcProxy.setBalancerBandwidth(null, req);
     } catch (ServiceException e) {
@@ -984,23 +941,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
         ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
   }
-  
+
   @Override
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
     try {
       GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
           null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
-     return rsp.hasDataEncryptionKey() ? 
+      return rsp.hasDataEncryptionKey() ?
           PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
 
   @Override
-  public boolean isFileClosed(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
+  public boolean isFileClosed(String src) throws IOException {
     IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
         .setSrc(src).build();
     try {
@@ -1030,7 +986,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
   @Override
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
@@ -1042,7 +998,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
   @Override
   public void allowSnapshot(String snapshotRoot) throws IOException {
     AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
@@ -1081,12 +1037,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
-    GetSnapshottableDirListingRequestProto req = 
+    GetSnapshottableDirListingRequestProto req =
         GetSnapshottableDirListingRequestProto.newBuilder().build();
     try {
       GetSnapshottableDirListingResponseProto result = rpcProxy
           .getSnapshottableDirListing(null, req);
-      
+
       if (result.hasSnapshottableDirList()) {
         return PBHelperClient.convert(result.getSnapshottableDirList());
       }
@@ -1103,9 +1059,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .newBuilder().setSnapshotRoot(snapshotRoot)
         .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
     try {
-      GetSnapshotDiffReportResponseProto result = 
+      GetSnapshotDiffReportResponseProto result =
           rpcProxy.getSnapshotDiffReport(null, req);
-    
+
       return PBHelperClient.convert(result.getDiffReport());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
@@ -1174,7 +1130,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     public int size() {
       return response.getElementsCount();
     }
-    
+
     @Override
     public boolean hasMore() {
       return response.getHasMore();
@@ -1182,19 +1138,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public BatchedEntries<CacheDirectiveEntry>
-      listCacheDirectives(long prevId,
-          CacheDirectiveInfo filter) throws IOException {
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
+      CacheDirectiveInfo filter) throws IOException {
     if (filter == null) {
       filter = new CacheDirectiveInfo.Builder().build();
     }
     try {
       return new BatchedCacheEntries(
-        rpcProxy.listCacheDirectives(null,
-          ListCacheDirectivesRequestProto.newBuilder().
-            setPrevId(prevId).
-            setFilter(PBHelperClient.convert(filter)).
-            build()));
+          rpcProxy.listCacheDirectives(null,
+              ListCacheDirectivesRequestProto.newBuilder().
+                  setPrevId(prevId).
+                  setFilter(PBHelperClient.convert(filter)).
+                  build()));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1202,7 +1157,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void addCachePool(CachePoolInfo info) throws IOException {
-    AddCachePoolRequestProto.Builder builder = 
+    AddCachePoolRequestProto.Builder builder =
         AddCachePoolRequestProto.newBuilder();
     builder.setInfo(PBHelperClient.convert(info));
     try {
@@ -1214,7 +1169,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void modifyCachePool(CachePoolInfo req) throws IOException {
-    ModifyCachePoolRequestProto.Builder builder = 
+    ModifyCachePoolRequestProto.Builder builder =
         ModifyCachePoolRequestProto.newBuilder();
     builder.setInfo(PBHelperClient.convert(req));
     try {
@@ -1227,22 +1182,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public void removeCachePool(String cachePoolName) throws IOException {
     try {
-      rpcProxy.removeCachePool(null, 
+      rpcProxy.removeCachePool(null,
           RemoveCachePoolRequestProto.newBuilder().
-            setPoolName(cachePoolName).build());
+              setPoolName(cachePoolName).build());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
   }
 
   private static class BatchedCachePoolEntries
-    implements BatchedEntries<CachePoolEntry> {
-      private final ListCachePoolsResponseProto proto;
-    
+      implements BatchedEntries<CachePoolEntry> {
+    private final ListCachePoolsResponseProto proto;
+
     public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
       this.proto = proto;
     }
-      
+
     @Override
     public CachePoolEntry get(int i) {
       CachePoolEntryProto elem = proto.getEntries(i);
@@ -1253,7 +1208,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     public int size() {
       return proto.getEntriesCount();
     }
-    
+
     @Override
     public boolean hasMore() {
       return proto.getHasMore();
@@ -1265,9 +1220,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
     try {
       return new BatchedCachePoolEntries(
-        rpcProxy.listCachePools(null,
-          ListCachePoolsRequestProto.newBuilder().
-            setPrevPoolName(prevKey).build()));
+          rpcProxy.listCachePools(null,
+              ListCachePoolsRequestProto.newBuilder().
+                  setPrevPoolName(prevKey).build()));
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1347,9 +1302,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
   @Override
   public void createEncryptionZone(String src, String keyName)
-    throws IOException {
+      throws IOException {
     final CreateEncryptionZoneRequestProto.Builder builder =
-      CreateEncryptionZoneRequestProto.newBuilder();
+        CreateEncryptionZoneRequestProto.newBuilder();
     builder.setSrc(src);
     if (keyName != null && !keyName.isEmpty()) {
       builder.setKeyName(keyName);
@@ -1363,8 +1318,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
-  public EncryptionZone getEZForPath(String src)
-      throws IOException {
+  public EncryptionZone getEZForPath(String src) throws IOException {
     final GetEZForPathRequestProto.Builder builder =
         GetEZForPathRequestProto.newBuilder();
     builder.setSrc(src);
@@ -1386,9 +1340,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
       throws IOException {
     final ListEncryptionZonesRequestProto req =
-      ListEncryptionZonesRequestProto.newBuilder()
-          .setId(id)
-          .build();
+        ListEncryptionZonesRequestProto.newBuilder()
+            .setId(id)
+            .build();
     try {
       EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
           rpcProxy.listEncryptionZones(null, req);
@@ -1397,8 +1351,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       for (EncryptionZoneProto p : response.getZonesList()) {
         elements.add(PBHelperClient.convert(p));
       }
-      return new BatchedListEntries<EncryptionZone>(elements,
-          response.getHasMore());
+      return new BatchedListEntries<>(elements, response.getHasMore());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -1418,7 +1371,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
   @Override
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
       throws IOException {
@@ -1434,11 +1387,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
-  
+
   @Override
-  public List<XAttr> listXAttrs(String src)
-      throws IOException {
-    ListXAttrsRequestProto.Builder builder = ListXAttrsRequestProto.newBuilder();
+  public List<XAttr> listXAttrs(String src) throws IOException {
+    ListXAttrsRequestProto.Builder builder =
+        ListXAttrsRequestProto.newBuilder();
     builder.setSrc(src);
     ListXAttrsRequestProto req = builder.build();
     try {

+ 324 - 320
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -207,34 +207,34 @@ public class PBHelperClient {
   public static ExtendedBlockProto convert(final ExtendedBlock b) {
     if (b == null) return null;
     return ExtendedBlockProto.newBuilder().
-      setPoolId(b.getBlockPoolId()).
-      setBlockId(b.getBlockId()).
-      setNumBytes(b.getNumBytes()).
-      setGenerationStamp(b.getGenerationStamp()).
-      build();
+        setPoolId(b.getBlockPoolId()).
+        setBlockId(b.getBlockId()).
+        setNumBytes(b.getNumBytes()).
+        setGenerationStamp(b.getGenerationStamp()).
+        build();
   }
 
   public static TokenProto convert(Token<?> tok) {
     return TokenProto.newBuilder().
-      setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
-      setPassword(ByteString.copyFrom(tok.getPassword())).
-      setKind(tok.getKind().toString()).
-      setService(tok.getService().toString()).build();
+        setIdentifier(ByteString.copyFrom(tok.getIdentifier())).
+        setPassword(ByteString.copyFrom(tok.getPassword())).
+        setKind(tok.getKind().toString()).
+        setService(tok.getService().toString()).build();
   }
 
   public static ShortCircuitShmIdProto convert(ShmId shmId) {
     return ShortCircuitShmIdProto.newBuilder().
-      setHi(shmId.getHi()).
-      setLo(shmId.getLo()).
-      build();
+        setHi(shmId.getHi()).
+        setLo(shmId.getLo()).
+        build();
 
   }
 
   public static ShortCircuitShmSlotProto convert(SlotId slotId) {
     return ShortCircuitShmSlotProto.newBuilder().
-      setShmId(convert(slotId.getShmId())).
-      setSlotIdx(slotId.getSlotIdx()).
-      build();
+        setShmId(convert(slotId.getShmId())).
+        setSlotIdx(slotId.getSlotIdx()).
+        build();
   }
 
   public static DatanodeIDProto convert(DatanodeID dn) {
@@ -242,23 +242,24 @@ public class PBHelperClient {
     // which is the same as the DatanodeUuid. Since StorageID is a required
     // field we pass the empty string if the DatanodeUuid is not yet known.
     return DatanodeIDProto.newBuilder()
-      .setIpAddr(dn.getIpAddr())
-      .setHostName(dn.getHostName())
-      .setXferPort(dn.getXferPort())
-      .setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "")
-      .setInfoPort(dn.getInfoPort())
-      .setInfoSecurePort(dn.getInfoSecurePort())
-      .setIpcPort(dn.getIpcPort()).build();
+        .setIpAddr(dn.getIpAddr())
+        .setHostName(dn.getHostName())
+        .setXferPort(dn.getXferPort())
+        .setDatanodeUuid(dn.getDatanodeUuid() != null ?
+            dn.getDatanodeUuid() : "")
+        .setInfoPort(dn.getInfoPort())
+        .setInfoSecurePort(dn.getInfoSecurePort())
+        .setIpcPort(dn.getIpcPort()).build();
   }
 
   public static DatanodeInfoProto.AdminState convert(
-    final DatanodeInfo.AdminStates inAs) {
+      final DatanodeInfo.AdminStates inAs) {
     switch (inAs) {
-      case NORMAL: return  DatanodeInfoProto.AdminState.NORMAL;
-      case DECOMMISSION_INPROGRESS:
-        return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
-      case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
-      default: return DatanodeInfoProto.AdminState.NORMAL;
+    case NORMAL: return  DatanodeInfoProto.AdminState.NORMAL;
+    case DECOMMISSION_INPROGRESS:
+      return DatanodeInfoProto.AdminState.DECOMMISSION_INPROGRESS;
+    case DECOMMISSIONED: return DatanodeInfoProto.AdminState.DECOMMISSIONED;
+    default: return DatanodeInfoProto.AdminState.NORMAL;
     }
   }
 
@@ -271,23 +272,23 @@ public class PBHelperClient {
       builder.setUpgradeDomain(info.getUpgradeDomain());
     }
     builder
-      .setId(convert((DatanodeID) info))
-      .setCapacity(info.getCapacity())
-      .setDfsUsed(info.getDfsUsed())
-      .setRemaining(info.getRemaining())
-      .setBlockPoolUsed(info.getBlockPoolUsed())
-      .setCacheCapacity(info.getCacheCapacity())
-      .setCacheUsed(info.getCacheUsed())
-      .setLastUpdate(info.getLastUpdate())
-      .setLastUpdateMonotonic(info.getLastUpdateMonotonic())
-      .setXceiverCount(info.getXceiverCount())
-      .setAdminState(convert(info.getAdminState()))
-      .build();
+        .setId(convert((DatanodeID) info))
+        .setCapacity(info.getCapacity())
+        .setDfsUsed(info.getDfsUsed())
+        .setRemaining(info.getRemaining())
+        .setBlockPoolUsed(info.getBlockPoolUsed())
+        .setCacheCapacity(info.getCacheCapacity())
+        .setCacheUsed(info.getCacheUsed())
+        .setLastUpdate(info.getLastUpdate())
+        .setLastUpdateMonotonic(info.getLastUpdateMonotonic())
+        .setXceiverCount(info.getXceiverCount())
+        .setAdminState(convert(info.getAdminState()))
+        .build();
     return builder.build();
   }
 
   public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
-    DatanodeInfo[] dnInfos) {
+      DatanodeInfo[] dnInfos) {
     return convert(dnInfos, 0);
   }
 
@@ -296,11 +297,11 @@ public class PBHelperClient {
    * {@code startIdx}.
    */
   public static List<? extends HdfsProtos.DatanodeInfoProto> convert(
-    DatanodeInfo[] dnInfos, int startIdx) {
+      DatanodeInfo[] dnInfos, int startIdx) {
     if (dnInfos == null)
       return null;
     ArrayList<HdfsProtos.DatanodeInfoProto> protos = Lists
-      .newArrayListWithCapacity(dnInfos.length);
+        .newArrayListWithCapacity(dnInfos.length);
     for (int i = startIdx; i < dnInfos.length; i++) {
       protos.add(convert(dnInfos[i]));
     }
@@ -337,48 +338,48 @@ public class PBHelperClient {
 
   public static StorageTypeProto convertStorageType(StorageType type) {
     switch(type) {
-      case DISK:
-        return StorageTypeProto.DISK;
-      case SSD:
-        return StorageTypeProto.SSD;
-      case ARCHIVE:
-        return StorageTypeProto.ARCHIVE;
-      case RAM_DISK:
-        return StorageTypeProto.RAM_DISK;
-      default:
-        throw new IllegalStateException(
+    case DISK:
+      return StorageTypeProto.DISK;
+    case SSD:
+      return StorageTypeProto.SSD;
+    case ARCHIVE:
+      return StorageTypeProto.ARCHIVE;
+    case RAM_DISK:
+      return StorageTypeProto.RAM_DISK;
+    default:
+      throw new IllegalStateException(
           "BUG: StorageType not found, type=" + type);
     }
   }
 
   public static StorageType convertStorageType(StorageTypeProto type) {
     switch(type) {
-      case DISK:
-        return StorageType.DISK;
-      case SSD:
-        return StorageType.SSD;
-      case ARCHIVE:
-        return StorageType.ARCHIVE;
-      case RAM_DISK:
-        return StorageType.RAM_DISK;
-      default:
-        throw new IllegalStateException(
+    case DISK:
+      return StorageType.DISK;
+    case SSD:
+      return StorageType.SSD;
+    case ARCHIVE:
+      return StorageType.ARCHIVE;
+    case RAM_DISK:
+      return StorageType.RAM_DISK;
+    default:
+      throw new IllegalStateException(
           "BUG: StorageTypeProto not found, type=" + type);
     }
   }
 
   public static List<StorageTypeProto> convertStorageTypes(
-    StorageType[] types) {
+      StorageType[] types) {
     return convertStorageTypes(types, 0);
   }
 
   public static List<StorageTypeProto> convertStorageTypes(
-    StorageType[] types, int startIdx) {
+      StorageType[] types, int startIdx) {
     if (types == null) {
       return null;
     }
     final List<StorageTypeProto> protos = new ArrayList<>(
-      types.length);
+        types.length);
     for (int i = startIdx; i < types.length; ++i) {
       protos.add(convertStorageType(types[i]));
     }
@@ -386,7 +387,7 @@ public class PBHelperClient {
   }
 
   public static InputStream vintPrefixed(final InputStream input)
-    throws IOException {
+      throws IOException {
     final int firstByte = input.read();
     if (firstByte == -1) {
       throw new EOFException("Premature EOF: no length prefix available");
@@ -438,8 +439,8 @@ public class PBHelperClient {
 
   public static HdfsProtos.CipherOptionProto convert(CipherOption option) {
     if (option != null) {
-      HdfsProtos.CipherOptionProto.Builder builder = HdfsProtos.CipherOptionProto.
-          newBuilder();
+      HdfsProtos.CipherOptionProto.Builder builder =
+          HdfsProtos.CipherOptionProto.newBuilder();
       if (option.getCipherSuite() != null) {
         builder.setSuite(convert(option.getCipherSuite()));
       }
@@ -514,7 +515,8 @@ public class PBHelperClient {
       storageIDs = null;
     } else {
       Preconditions.checkState(storageIDsCount == locs.size());
-      storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
+      storageIDs = proto.getStorageIDsList()
+          .toArray(new String[storageIDsCount]);
     }
 
     // Set values from the isCached list, re-using references from loc
@@ -550,7 +552,7 @@ public class PBHelperClient {
       List<StorageTypeProto> storageTypesList, int expectedSize) {
     final StorageType[] storageTypes = new StorageType[expectedSize];
     if (storageTypesList.size() != expectedSize) {
-     // missing storage types
+      // missing storage types
       Preconditions.checkState(storageTypesList.isEmpty());
       Arrays.fill(storageTypes, StorageType.DEFAULT);
     } else {
@@ -570,9 +572,9 @@ public class PBHelperClient {
 
   // DatanodeId
   public static DatanodeID convert(DatanodeIDProto dn) {
-    return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
-        dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
-        .getInfoSecurePort() : 0, dn.getIpcPort());
+    return new DatanodeID(dn.getIpAddr(), dn.getHostName(),
+        dn.getDatanodeUuid(), dn.getXferPort(), dn.getInfoPort(),
+        dn.hasInfoSecurePort() ? dn.getInfoSecurePort() : 0, dn.getIpcPort());
   }
 
   public static AdminStates convert(AdminState adminState) {
@@ -611,8 +613,8 @@ public class PBHelperClient {
     return policies;
   }
 
-  public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws
-    IOException {
+  public static EventBatchList convert(GetEditsFromTxidResponseProto resp)
+      throws IOException {
     final InotifyProtos.EventsListProto list = resp.getEventsList();
     final long firstTxid = list.getFirstTxid();
     final long lastTxid = list.getLastTxid();
@@ -631,82 +633,82 @@ public class PBHelperClient {
       List<Event> events = Lists.newArrayList();
       for (InotifyProtos.EventProto p : bp.getEventsList()) {
         switch (p.getType()) {
-          case EVENT_CLOSE:
-            InotifyProtos.CloseEventProto close =
-                InotifyProtos.CloseEventProto.parseFrom(p.getContents());
-            events.add(new Event.CloseEvent(close.getPath(),
-                close.getFileSize(), close.getTimestamp()));
-            break;
-          case EVENT_CREATE:
-            InotifyProtos.CreateEventProto create =
-                InotifyProtos.CreateEventProto.parseFrom(p.getContents());
-            events.add(new Event.CreateEvent.Builder()
-                .iNodeType(createTypeConvert(create.getType()))
-                .path(create.getPath())
-                .ctime(create.getCtime())
-                .ownerName(create.getOwnerName())
-                .groupName(create.getGroupName())
-                .perms(convert(create.getPerms()))
-                .replication(create.getReplication())
-                .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
-                    create.getSymlinkTarget())
-                .defaultBlockSize(create.getDefaultBlockSize())
-                .overwrite(create.getOverwrite()).build());
-            break;
-          case EVENT_METADATA:
-            InotifyProtos.MetadataUpdateEventProto meta =
-                InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
-            events.add(new Event.MetadataUpdateEvent.Builder()
-                .path(meta.getPath())
-                .metadataType(metadataUpdateTypeConvert(meta.getType()))
-                .mtime(meta.getMtime())
-                .atime(meta.getAtime())
-                .replication(meta.getReplication())
-                .ownerName(
-                    meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
-                .groupName(
-                    meta.getGroupName().isEmpty() ? null : meta.getGroupName())
-                .perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
-                .acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
-                    meta.getAclsList()))
-                .xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
-                    meta.getXAttrsList()))
-                .xAttrsRemoved(meta.getXAttrsRemoved())
-                .build());
-            break;
-          case EVENT_RENAME:
-            InotifyProtos.RenameEventProto rename =
-                InotifyProtos.RenameEventProto.parseFrom(p.getContents());
-            events.add(new Event.RenameEvent.Builder()
-                  .srcPath(rename.getSrcPath())
-                  .dstPath(rename.getDestPath())
-                  .timestamp(rename.getTimestamp())
-                  .build());
-            break;
-          case EVENT_APPEND:
-            InotifyProtos.AppendEventProto append =
-                InotifyProtos.AppendEventProto.parseFrom(p.getContents());
-            events.add(new Event.AppendEvent.Builder().path(append.getPath())
-                .newBlock(append.hasNewBlock() && append.getNewBlock())
-                .build());
-            break;
-          case EVENT_UNLINK:
-            InotifyProtos.UnlinkEventProto unlink =
-                InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
-            events.add(new Event.UnlinkEvent.Builder()
-                  .path(unlink.getPath())
-                  .timestamp(unlink.getTimestamp())
-                  .build());
-            break;
-          case EVENT_TRUNCATE:
-            InotifyProtos.TruncateEventProto truncate =
-                InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
-            events.add(new Event.TruncateEvent(truncate.getPath(),
-                truncate.getFileSize(), truncate.getTimestamp()));
-            break;
-          default:
-            throw new RuntimeException("Unexpected inotify event type: " +
-                p.getType());
+        case EVENT_CLOSE:
+          InotifyProtos.CloseEventProto close =
+              InotifyProtos.CloseEventProto.parseFrom(p.getContents());
+          events.add(new Event.CloseEvent(close.getPath(),
+              close.getFileSize(), close.getTimestamp()));
+          break;
+        case EVENT_CREATE:
+          InotifyProtos.CreateEventProto create =
+              InotifyProtos.CreateEventProto.parseFrom(p.getContents());
+          events.add(new Event.CreateEvent.Builder()
+              .iNodeType(createTypeConvert(create.getType()))
+              .path(create.getPath())
+              .ctime(create.getCtime())
+              .ownerName(create.getOwnerName())
+              .groupName(create.getGroupName())
+              .perms(convert(create.getPerms()))
+              .replication(create.getReplication())
+              .symlinkTarget(create.getSymlinkTarget().isEmpty() ? null :
+                  create.getSymlinkTarget())
+              .defaultBlockSize(create.getDefaultBlockSize())
+              .overwrite(create.getOverwrite()).build());
+          break;
+        case EVENT_METADATA:
+          InotifyProtos.MetadataUpdateEventProto meta =
+              InotifyProtos.MetadataUpdateEventProto.parseFrom(p.getContents());
+          events.add(new Event.MetadataUpdateEvent.Builder()
+              .path(meta.getPath())
+              .metadataType(metadataUpdateTypeConvert(meta.getType()))
+              .mtime(meta.getMtime())
+              .atime(meta.getAtime())
+              .replication(meta.getReplication())
+              .ownerName(
+                  meta.getOwnerName().isEmpty() ? null : meta.getOwnerName())
+              .groupName(
+                  meta.getGroupName().isEmpty() ? null : meta.getGroupName())
+              .perms(meta.hasPerms() ? convert(meta.getPerms()) : null)
+              .acls(meta.getAclsList().isEmpty() ? null : convertAclEntry(
+                  meta.getAclsList()))
+              .xAttrs(meta.getXAttrsList().isEmpty() ? null : convertXAttrs(
+                  meta.getXAttrsList()))
+              .xAttrsRemoved(meta.getXAttrsRemoved())
+              .build());
+          break;
+        case EVENT_RENAME:
+          InotifyProtos.RenameEventProto rename =
+              InotifyProtos.RenameEventProto.parseFrom(p.getContents());
+          events.add(new Event.RenameEvent.Builder()
+              .srcPath(rename.getSrcPath())
+              .dstPath(rename.getDestPath())
+              .timestamp(rename.getTimestamp())
+              .build());
+          break;
+        case EVENT_APPEND:
+          InotifyProtos.AppendEventProto append =
+              InotifyProtos.AppendEventProto.parseFrom(p.getContents());
+          events.add(new Event.AppendEvent.Builder().path(append.getPath())
+              .newBlock(append.hasNewBlock() && append.getNewBlock())
+              .build());
+          break;
+        case EVENT_UNLINK:
+          InotifyProtos.UnlinkEventProto unlink =
+              InotifyProtos.UnlinkEventProto.parseFrom(p.getContents());
+          events.add(new Event.UnlinkEvent.Builder()
+              .path(unlink.getPath())
+              .timestamp(unlink.getTimestamp())
+              .build());
+          break;
+        case EVENT_TRUNCATE:
+          InotifyProtos.TruncateEventProto truncate =
+              InotifyProtos.TruncateEventProto.parseFrom(p.getContents());
+          events.add(new Event.TruncateEvent(truncate.getPath(),
+              truncate.getFileSize(), truncate.getTimestamp()));
+          break;
+        default:
+          throw new RuntimeException("Unexpected inotify event type: " +
+              p.getType());
         }
       }
       batches.add(new EventBatch(txid, events.toArray(new Event[0])));
@@ -878,7 +880,7 @@ public class PBHelperClient {
   }
 
   static InotifyProtos.INodeType createTypeConvert(Event.CreateEvent.INodeType
-                                                       type) {
+      type) {
     switch (type) {
     case DIRECTORY:
       return InotifyProtos.INodeType.I_TYPE_DIRECTORY;
@@ -1090,7 +1092,7 @@ public class PBHelperClient {
     String poolName = checkNotNull(proto.getPoolName());
     CachePoolInfo info = new CachePoolInfo(poolName);
     if (proto.hasOwnerName()) {
-        info.setOwnerName(proto.getOwnerName());
+      info.setOwnerName(proto.getOwnerName());
     }
     if (proto.hasGroupName()) {
       info.setGroupName(proto.getGroupName());
@@ -1138,8 +1140,7 @@ public class PBHelperClient {
     return builder.build();
   }
 
-  public static CacheDirectiveInfoProto convert
-      (CacheDirectiveInfo info) {
+  public static CacheDirectiveInfoProto convert(CacheDirectiveInfo info) {
     CacheDirectiveInfoProto.Builder builder =
         CacheDirectiveInfoProto.newBuilder();
     if (info.getId() != null) {
@@ -1184,10 +1185,8 @@ public class PBHelperClient {
     return builder.build();
   }
 
-  public static CacheDirectiveInfo convert
-      (CacheDirectiveInfoProto proto) {
-    CacheDirectiveInfo.Builder builder =
-        new CacheDirectiveInfo.Builder();
+  public static CacheDirectiveInfo convert(CacheDirectiveInfoProto proto) {
+    CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder();
     if (proto.hasId()) {
       builder.setId(proto.getId());
     }
@@ -1223,7 +1222,8 @@ public class PBHelperClient {
     return value;
   }
 
-  public static SnapshotDiffReport convert(SnapshotDiffReportProto reportProto) {
+  public static SnapshotDiffReport convert(
+      SnapshotDiffReportProto reportProto) {
     if (reportProto == null) {
       return null;
     }
@@ -1442,8 +1442,7 @@ public class PBHelperClient {
     }
   }
 
-  public static SafeModeActionProto convert(
-      SafeModeAction a) {
+  public static SafeModeActionProto convert(SafeModeAction a) {
     switch (a) {
     case SAFEMODE_LEAVE:
       return SafeModeActionProto.SAFEMODE_LEAVE;
@@ -1469,16 +1468,18 @@ public class PBHelperClient {
     result[ClientProtocol.GET_STATS_CAPACITY_IDX] = res.getCapacity();
     result[ClientProtocol.GET_STATS_USED_IDX] = res.getUsed();
     result[ClientProtocol.GET_STATS_REMAINING_IDX] = res.getRemaining();
-    result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] = res.getUnderReplicated();
-    result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] = res.getCorruptBlocks();
-    result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] = res.getMissingBlocks();
+    result[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX] =
+        res.getUnderReplicated();
+    result[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX] =
+        res.getCorruptBlocks();
+    result[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX] =
+        res.getMissingBlocks();
     result[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX] =
         res.getMissingReplOneBlocks();
     return result;
   }
 
-  public static DatanodeReportTypeProto
-    convert(DatanodeReportType t) {
+  public static DatanodeReportTypeProto convert(DatanodeReportType t) {
     switch (t) {
     case ALL: return DatanodeReportTypeProto.ALL;
     case LIVE: return DatanodeReportTypeProto.LIVE;
@@ -1636,8 +1637,8 @@ public class PBHelperClient {
       DatanodeStorageReport[] reports) {
     final List<DatanodeStorageReportProto> protos
         = new ArrayList<>(reports.length);
-    for(int i = 0; i < reports.length; i++) {
-      protos.add(convertDatanodeStorageReport(reports[i]));
+    for (DatanodeStorageReport report : reports) {
+      protos.add(convertDatanodeStorageReport(report));
     }
     return protos;
   }
@@ -1682,20 +1683,20 @@ public class PBHelperClient {
   public static FsServerDefaultsProto convert(FsServerDefaults fs) {
     if (fs == null) return null;
     return FsServerDefaultsProto.newBuilder().
-      setBlockSize(fs.getBlockSize()).
-      setBytesPerChecksum(fs.getBytesPerChecksum()).
-      setWritePacketSize(fs.getWritePacketSize())
-      .setReplication(fs.getReplication())
-      .setFileBufferSize(fs.getFileBufferSize())
-      .setEncryptDataTransfer(fs.getEncryptDataTransfer())
-      .setTrashInterval(fs.getTrashInterval())
-      .setChecksumType(convert(fs.getChecksumType()))
-      .build();
+        setBlockSize(fs.getBlockSize()).
+        setBytesPerChecksum(fs.getBytesPerChecksum()).
+        setWritePacketSize(fs.getWritePacketSize())
+        .setReplication(fs.getReplication())
+        .setFileBufferSize(fs.getFileBufferSize())
+        .setEncryptDataTransfer(fs.getEncryptDataTransfer())
+        .setTrashInterval(fs.getTrashInterval())
+        .setChecksumType(convert(fs.getChecksumType()))
+        .build();
   }
 
   public static EnumSetWritable<CreateFlag> convertCreateFlag(int flag) {
     EnumSet<CreateFlag> result =
-       EnumSet.noneOf(CreateFlag.class);
+        EnumSet.noneOf(CreateFlag.class);
     if ((flag & CreateFlagProto.APPEND_VALUE) == CreateFlagProto.APPEND_VALUE) {
       result.add(CreateFlag.APPEND);
     }
@@ -1714,7 +1715,7 @@ public class PBHelperClient {
         == CreateFlagProto.NEW_BLOCK_VALUE) {
       result.add(CreateFlag.NEW_BLOCK);
     }
-    return new EnumSetWritable<CreateFlag>(result, CreateFlag.class);
+    return new EnumSetWritable<>(result, CreateFlag.class);
   }
 
   public static EnumSet<CacheFlag> convertCacheFlags(int flags) {
@@ -1736,20 +1737,20 @@ public class PBHelperClient {
     }
 
     HdfsFileStatusProto.Builder builder =
-     HdfsFileStatusProto.newBuilder().
-      setLength(fs.getLen()).
-      setFileType(fType).
-      setBlockReplication(fs.getReplication()).
-      setBlocksize(fs.getBlockSize()).
-      setModificationTime(fs.getModificationTime()).
-      setAccessTime(fs.getAccessTime()).
-      setPermission(convert(fs.getPermission())).
-      setOwner(fs.getOwner()).
-      setGroup(fs.getGroup()).
-      setFileId(fs.getFileId()).
-      setChildrenNum(fs.getChildrenNum()).
-      setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
-      setStoragePolicy(fs.getStoragePolicy());
+        HdfsFileStatusProto.newBuilder().
+            setLength(fs.getLen()).
+            setFileType(fType).
+            setBlockReplication(fs.getReplication()).
+            setBlocksize(fs.getBlockSize()).
+            setModificationTime(fs.getModificationTime()).
+            setAccessTime(fs.getAccessTime()).
+            setPermission(convert(fs.getPermission())).
+            setOwner(fs.getOwner()).
+            setGroup(fs.getGroup()).
+            setFileId(fs.getFileId()).
+            setChildrenNum(fs.getChildrenNum()).
+            setPath(ByteString.copyFrom(fs.getLocalNameInBytes())).
+            setStoragePolicy(fs.getStoragePolicy());
     if (fs.isSymlink())  {
       builder.setSymlink(ByteString.copyFrom(fs.getSymlinkInBytes()));
     }
@@ -1779,9 +1780,11 @@ public class PBHelperClient {
     HdfsFileStatusProto fs = convert(status.getDirStatus());
     SnapshottableDirectoryStatusProto.Builder builder =
         SnapshottableDirectoryStatusProto
-        .newBuilder().setSnapshotNumber(snapshotNumber)
-        .setSnapshotQuota(snapshotQuota).setParentFullpath(parentFullPathBytes)
-        .setDirStatus(fs);
+            .newBuilder()
+            .setSnapshotNumber(snapshotNumber)
+            .setSnapshotQuota(snapshotQuota)
+            .setParentFullpath(parentFullPathBytes)
+            .setDirStatus(fs);
     return builder.build();
   }
 
@@ -1816,14 +1819,15 @@ public class PBHelperClient {
       result.setRemaining(fsStats[ClientProtocol.GET_STATS_REMAINING_IDX]);
     if (fsStats.length >= ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX + 1)
       result.setUnderReplicated(
-              fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
+          fsStats[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX]);
     if (fsStats.length >= ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX + 1)
       result.setCorruptBlocks(
           fsStats[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX]);
     if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX + 1)
       result.setMissingBlocks(
           fsStats[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX]);
-    if (fsStats.length >= ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
+    if (fsStats.length >=
+        ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX + 1)
       result.setMissingReplOneBlocks(
           fsStats[ClientProtocol.GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX]);
     return result.build();
@@ -1901,7 +1905,7 @@ public class PBHelperClient {
   public static ContentSummaryProto convert(ContentSummary cs) {
     if (cs == null) return null;
     ContentSummaryProto.Builder builder = ContentSummaryProto.newBuilder();
-        builder.setLength(cs.getLength()).
+    builder.setLength(cs.getLength()).
         setFileCount(cs.getFileCount()).
         setDirectoryCount(cs.getDirectoryCount()).
         setQuota(cs.getQuota()).
@@ -1951,11 +1955,11 @@ public class PBHelperClient {
     return builder.build();
   }
 
-  public static List<StorageReportProto> convertStorageReports(StorageReport[] storages) {
-    final List<StorageReportProto> protos = new ArrayList<StorageReportProto>(
-        storages.length);
-    for(int i = 0; i < storages.length; i++) {
-      protos.add(convert(storages[i]));
+  public static List<StorageReportProto> convertStorageReports(
+      StorageReport[] storages) {
+    final List<StorageReportProto> protos = new ArrayList<>(storages.length);
+    for (StorageReport storage : storages) {
+      protos.add(convert(storage));
     }
     return protos;
   }
@@ -1978,17 +1982,16 @@ public class PBHelperClient {
     if (entry == null) {
       return null;
     }
-    ByteString sourcePath = ByteString
-        .copyFrom(entry.getSourcePath() == null ? DFSUtilClient.EMPTY_BYTES : entry
-            .getSourcePath());
+    ByteString sourcePath = ByteString.copyFrom(entry.getSourcePath() == null ?
+        DFSUtilClient.EMPTY_BYTES : entry.getSourcePath());
     String modification = entry.getType().getLabel();
     SnapshotDiffReportEntryProto.Builder builder = SnapshotDiffReportEntryProto
         .newBuilder().setFullpath(sourcePath)
         .setModificationLabel(modification);
     if (entry.getType() == DiffType.RENAME) {
-      ByteString targetPath = ByteString
-          .copyFrom(entry.getTargetPath() == null ? DFSUtilClient.EMPTY_BYTES : entry
-              .getTargetPath());
+      ByteString targetPath =
+          ByteString.copyFrom(entry.getTargetPath() == null ?
+              DFSUtilClient.EMPTY_BYTES : entry.getTargetPath());
       builder.setTargetPath(targetPath);
     }
     return builder.build();
@@ -2006,12 +2009,11 @@ public class PBHelperClient {
         entryProtos.add(entryProto);
     }
 
-    SnapshotDiffReportProto reportProto = SnapshotDiffReportProto.newBuilder()
+    return SnapshotDiffReportProto.newBuilder()
         .setSnapshotRoot(report.getSnapshotRoot())
         .setFromSnapshot(report.getFromSnapshot())
         .setToSnapshot(report.getLaterSnapshotName())
         .addAllDiffReportEntries(entryProtos).build();
-    return reportProto;
   }
 
   public static CacheDirectiveStatsProto convert(CacheDirectiveStats stats) {
@@ -2034,7 +2036,7 @@ public class PBHelperClient {
   }
 
   public static boolean[] convertBooleanList(
-    List<Boolean> targetPinningsList) {
+      List<Boolean> targetPinningsList) {
     final boolean[] targetPinnings = new boolean[targetPinningsList.size()];
     for (int i = 0; i < targetPinningsList.size(); i++) {
       targetPinnings[i] = targetPinningsList.get(i);
@@ -2060,7 +2062,8 @@ public class PBHelperClient {
   }
 
   public static DatanodeLocalInfoProto convert(DatanodeLocalInfo info) {
-    DatanodeLocalInfoProto.Builder builder = DatanodeLocalInfoProto.newBuilder();
+    DatanodeLocalInfoProto.Builder builder =
+        DatanodeLocalInfoProto.newBuilder();
     builder.setSoftwareVersion(info.getSoftwareVersion());
     builder.setConfigVersion(info.getConfigVersion());
     builder.setUptime(info.getUptime());
@@ -2116,9 +2119,9 @@ public class PBHelperClient {
   }
 
   public static ListXAttrsResponseProto convertListXAttrsResponse(
-    List<XAttr> names) {
+      List<XAttr> names) {
     ListXAttrsResponseProto.Builder builder =
-      ListXAttrsResponseProto.newBuilder();
+        ListXAttrsResponseProto.newBuilder();
     if (names != null) {
       builder.addAllXAttrs(convertXAttrProto(names));
     }
@@ -2140,114 +2143,115 @@ public class PBHelperClient {
         slotId.getSlotIdx());
   }
 
-  public static GetEditsFromTxidResponseProto convertEditsResponse(EventBatchList el) {
+  public static GetEditsFromTxidResponseProto convertEditsResponse(
+      EventBatchList el) {
     InotifyProtos.EventsListProto.Builder builder =
         InotifyProtos.EventsListProto.newBuilder();
     for (EventBatch b : el.getBatches()) {
       List<InotifyProtos.EventProto> events = Lists.newArrayList();
       for (Event e : b.getEvents()) {
         switch (e.getEventType()) {
-          case CLOSE:
-            Event.CloseEvent ce = (Event.CloseEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_CLOSE)
-                .setContents(
-                    InotifyProtos.CloseEventProto.newBuilder()
-                        .setPath(ce.getPath())
-                        .setFileSize(ce.getFileSize())
-                        .setTimestamp(ce.getTimestamp()).build().toByteString()
-                ).build());
-            break;
-          case CREATE:
-            Event.CreateEvent ce2 = (Event.CreateEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_CREATE)
-                .setContents(
-                    InotifyProtos.CreateEventProto.newBuilder()
-                        .setType(createTypeConvert(ce2.getiNodeType()))
-                        .setPath(ce2.getPath())
-                        .setCtime(ce2.getCtime())
-                        .setOwnerName(ce2.getOwnerName())
-                        .setGroupName(ce2.getGroupName())
-                        .setPerms(convert(ce2.getPerms()))
-                        .setReplication(ce2.getReplication())
-                        .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
-                            "" : ce2.getSymlinkTarget())
-                        .setDefaultBlockSize(ce2.getDefaultBlockSize())
-                        .setOverwrite(ce2.getOverwrite()).build().toByteString()
-                ).build());
-            break;
-          case METADATA:
-            Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
-            InotifyProtos.MetadataUpdateEventProto.Builder metaB =
-                InotifyProtos.MetadataUpdateEventProto.newBuilder()
-                    .setPath(me.getPath())
-                    .setType(metadataUpdateTypeConvert(me.getMetadataType()))
-                    .setMtime(me.getMtime())
-                    .setAtime(me.getAtime())
-                    .setReplication(me.getReplication())
-                    .setOwnerName(me.getOwnerName() == null ? "" :
-                        me.getOwnerName())
-                    .setGroupName(me.getGroupName() == null ? "" :
-                        me.getGroupName())
-                    .addAllAcls(me.getAcls() == null ?
-                        Lists.<AclEntryProto>newArrayList() :
-                        convertAclEntryProto(me.getAcls()))
-                    .addAllXAttrs(me.getxAttrs() == null ?
-                        Lists.<XAttrProto>newArrayList() :
-                        convertXAttrProto(me.getxAttrs()))
-                    .setXAttrsRemoved(me.isxAttrsRemoved());
-            if (me.getPerms() != null) {
-              metaB.setPerms(convert(me.getPerms()));
-            }
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_METADATA)
-                .setContents(metaB.build().toByteString())
-                .build());
-            break;
-          case RENAME:
-            Event.RenameEvent re = (Event.RenameEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_RENAME)
-                .setContents(
-                    InotifyProtos.RenameEventProto.newBuilder()
-                        .setSrcPath(re.getSrcPath())
-                        .setDestPath(re.getDstPath())
-                        .setTimestamp(re.getTimestamp()).build().toByteString()
-                ).build());
-            break;
-          case APPEND:
-            Event.AppendEvent re2 = (Event.AppendEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_APPEND)
-                .setContents(InotifyProtos.AppendEventProto.newBuilder()
-                    .setPath(re2.getPath())
-                    .setNewBlock(re2.toNewBlock()).build().toByteString())
-                .build());
-            break;
-          case UNLINK:
-            Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_UNLINK)
-                .setContents(
-                    InotifyProtos.UnlinkEventProto.newBuilder()
-                        .setPath(ue.getPath())
-                        .setTimestamp(ue.getTimestamp()).build().toByteString()
-                ).build());
-            break;
-          case TRUNCATE:
-            Event.TruncateEvent te = (Event.TruncateEvent) e;
-            events.add(InotifyProtos.EventProto.newBuilder()
-                .setType(InotifyProtos.EventType.EVENT_TRUNCATE)
-                .setContents(
-                    InotifyProtos.TruncateEventProto.newBuilder()
-                        .setPath(te.getPath())
-                        .setFileSize(te.getFileSize())
-                        .setTimestamp(te.getTimestamp()).build().toByteString()
-                ).build());
-            break;
-          default:
-            throw new RuntimeException("Unexpected inotify event: " + e);
+        case CLOSE:
+          Event.CloseEvent ce = (Event.CloseEvent) e;
+          events.add(InotifyProtos.EventProto.newBuilder()
+              .setType(InotifyProtos.EventType.EVENT_CLOSE)
+              .setContents(
+                  InotifyProtos.CloseEventProto.newBuilder()
+                      .setPath(ce.getPath())
+                      .setFileSize(ce.getFileSize())
+                      .setTimestamp(ce.getTimestamp()).build().toByteString()
+              ).build());
+          break;
+        case CREATE:
+          Event.CreateEvent ce2 = (Event.CreateEvent) e;
+          events.add(InotifyProtos.EventProto.newBuilder()
+              .setType(InotifyProtos.EventType.EVENT_CREATE)
+              .setContents(
+                  InotifyProtos.CreateEventProto.newBuilder()
+                      .setType(createTypeConvert(ce2.getiNodeType()))
+                      .setPath(ce2.getPath())
+                      .setCtime(ce2.getCtime())
+                      .setOwnerName(ce2.getOwnerName())
+                      .setGroupName(ce2.getGroupName())
+                      .setPerms(convert(ce2.getPerms()))
+                      .setReplication(ce2.getReplication())
+                      .setSymlinkTarget(ce2.getSymlinkTarget() == null ?
+                          "" : ce2.getSymlinkTarget())
+                      .setDefaultBlockSize(ce2.getDefaultBlockSize())
+                      .setOverwrite(ce2.getOverwrite()).build().toByteString()
+              ).build());
+          break;
+        case METADATA:
+          Event.MetadataUpdateEvent me = (Event.MetadataUpdateEvent) e;
+          InotifyProtos.MetadataUpdateEventProto.Builder metaB =
+              InotifyProtos.MetadataUpdateEventProto.newBuilder()
+                  .setPath(me.getPath())
+                  .setType(metadataUpdateTypeConvert(me.getMetadataType()))
+                  .setMtime(me.getMtime())
+                  .setAtime(me.getAtime())
+                  .setReplication(me.getReplication())
+                  .setOwnerName(me.getOwnerName() == null ? "" :
+                      me.getOwnerName())
+                  .setGroupName(me.getGroupName() == null ? "" :
+                      me.getGroupName())
+                  .addAllAcls(me.getAcls() == null ?
+                      Lists.<AclEntryProto>newArrayList() :
+                      convertAclEntryProto(me.getAcls()))
+                  .addAllXAttrs(me.getxAttrs() == null ?
+                      Lists.<XAttrProto>newArrayList() :
+                      convertXAttrProto(me.getxAttrs()))
+                  .setXAttrsRemoved(me.isxAttrsRemoved());
+          if (me.getPerms() != null) {
+            metaB.setPerms(convert(me.getPerms()));
+          }
+          events.add(InotifyProtos.EventProto.newBuilder()
+              .setType(InotifyProtos.EventType.EVENT_METADATA)
+              .setContents(metaB.build().toByteString())
+              .build());
+          break;
+        case RENAME:
+          Event.RenameEvent re = (Event.RenameEvent) e;
+          events.add(InotifyProtos.EventProto.newBuilder()
+              .setType(InotifyProtos.EventType.EVENT_RENAME)
+              .setContents(
+                  InotifyProtos.RenameEventProto.newBuilder()
+                      .setSrcPath(re.getSrcPath())
+                      .setDestPath(re.getDstPath())
+                      .setTimestamp(re.getTimestamp()).build().toByteString()
+              ).build());
+          break;
+        case APPEND:
+          Event.AppendEvent re2 = (Event.AppendEvent) e;
+          events.add(InotifyProtos.EventProto.newBuilder()
+              .setType(InotifyProtos.EventType.EVENT_APPEND)
+              .setContents(InotifyProtos.AppendEventProto.newBuilder()
+                  .setPath(re2.getPath())
+                  .setNewBlock(re2.toNewBlock()).build().toByteString())
+              .build());
+          break;
+        case UNLINK:
+          Event.UnlinkEvent ue = (Event.UnlinkEvent) e;
+          events.add(InotifyProtos.EventProto.newBuilder()
+              .setType(InotifyProtos.EventType.EVENT_UNLINK)
+              .setContents(
+                  InotifyProtos.UnlinkEventProto.newBuilder()
+                      .setPath(ue.getPath())
+                      .setTimestamp(ue.getTimestamp()).build().toByteString()
+              ).build());
+          break;
+        case TRUNCATE:
+          Event.TruncateEvent te = (Event.TruncateEvent) e;
+          events.add(InotifyProtos.EventProto.newBuilder()
+              .setType(InotifyProtos.EventType.EVENT_TRUNCATE)
+              .setContents(
+                  InotifyProtos.TruncateEventProto.newBuilder()
+                      .setPath(te.getPath())
+                      .setFileSize(te.getFileSize())
+                      .setTimestamp(te.getTimestamp()).build().toByteString()
+              ).build());
+          break;
+        default:
+          throw new RuntimeException("Unexpected inotify event: " + e);
         }
       }
       builder.addBatch(InotifyProtos.EventBatchProto.newBuilder().

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java

@@ -34,7 +34,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti
 @InterfaceAudience.Private
 public class DelegationTokenIdentifier
     extends AbstractDelegationTokenIdentifier {
-  public static final Text HDFS_DELEGATION_KIND = new Text("HDFS_DELEGATION_TOKEN");
+  public static final Text HDFS_DELEGATION_KIND =
+      new Text("HDFS_DELEGATION_TOKEN");
 
   /**
    * Create an empty delegation token identifier for reading into.

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java

@@ -39,7 +39,7 @@ public class DelegationTokenSelector
 
   /**
    * Select the delegation token for hdfs.  The port will be rewritten to
-   * the port of hdfs.service.host_$nnAddr, or the default rpc namenode port. 
+   * the port of hdfs.service.host_$nnAddr, or the default rpc namenode port.
    * This method should only be called by non-hdfs filesystems that do not
    * use the rpc port to acquire tokens.  Ex. webhdfs, hftp 
    * @param nnUri of the remote namenode
@@ -56,15 +56,15 @@ public class DelegationTokenSelector
     // and correctly determine the value
     Text serviceName = SecurityUtil.buildTokenService(nnUri);
     final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
-    
+
     int nnRpcPort = HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
     if (nnServiceName != null) {
-      nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
+      nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort();
     }
     // use original hostname from the uri to avoid unintentional host resolving
     serviceName = SecurityUtil.buildTokenService(
     		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
-    
+
     return selectToken(serviceName, tokens);
   }
 

+ 17 - 15
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -52,11 +52,11 @@ public class BlockMetadataHeader {
       BlockMetadataHeader.class);
 
   public static final short VERSION = 1;
-  
+
   /**
    * Header includes everything except the checksum(s) themselves.
    * Version is two bytes. Following it is the DataChecksum
-   * that occupies 5 bytes. 
+   * that occupies 5 bytes.
    */
   private final short version;
   private DataChecksum checksum = null;
@@ -66,7 +66,7 @@ public class BlockMetadataHeader {
     this.checksum = checksum;
     this.version = version;
   }
-  
+
   /** Get the version */
   public short getVersion() {
     return version;
@@ -137,13 +137,14 @@ public class BlockMetadataHeader {
    * @return Metadata Header
    * @throws IOException
    */
-  public static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
+  public static BlockMetadataHeader readHeader(DataInputStream in)
+      throws IOException {
     return readHeader(in.readShort(), in);
   }
-  
+
   /**
    * Reads header at the top of metadata file and returns the header.
-   * 
+   *
    * @return metadata header for the block
    * @throws IOException
    */
@@ -157,39 +158,40 @@ public class BlockMetadataHeader {
       IOUtils.closeStream(in);
     }
   }
-  
+
   /**
    * Read the header at the beginning of the given block meta file.
    * The current file position will be altered by this method.
    * If an error occurs, the file is <em>not</em> closed.
    */
-  public static BlockMetadataHeader readHeader(RandomAccessFile raf) throws IOException {
+  public static BlockMetadataHeader readHeader(RandomAccessFile raf)
+      throws IOException {
     byte[] buf = new byte[getHeaderSize()];
     raf.seek(0);
     raf.readFully(buf, 0, buf.length);
     return readHeader(new DataInputStream(new ByteArrayInputStream(buf)));
   }
-  
+
   // Version is already read.
-  private static BlockMetadataHeader readHeader(short version, DataInputStream in) 
-                                   throws IOException {
+  private static BlockMetadataHeader readHeader(short version,
+      DataInputStream in) throws IOException {
     DataChecksum checksum = DataChecksum.newDataChecksum(in);
     return new BlockMetadataHeader(version, checksum);
   }
-  
+
   /**
    * This writes all the fields till the beginning of checksum.
    * @param out DataOutputStream
    * @throws IOException
    */
   @VisibleForTesting
-  public static void writeHeader(DataOutputStream out, 
-                                  BlockMetadataHeader header) 
+  public static void writeHeader(DataOutputStream out,
+                                  BlockMetadataHeader header)
                                   throws IOException {
     out.writeShort(header.getVersion());
     header.getChecksum().writeHeader(out);
   }
-  
+
   /**
    * Writes all the fields till the beginning of checksum.
    * @throws IOException on error

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/CachingStrategy.java

@@ -23,7 +23,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 public class CachingStrategy {
   private final Boolean dropBehind; // null = use server defaults
   private final Long readahead; // null = use server defaults
-  
+
   public static CachingStrategy newDefaultStrategy() {
     return new CachingStrategy(null, null);
   }
@@ -64,7 +64,7 @@ public class CachingStrategy {
   public Boolean getDropBehind() {
     return dropBehind;
   }
-  
+
   public Long getReadahead() {
     return readahead;
   }

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java

@@ -25,19 +25,21 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 
 /**
  * Exception indicating that DataNode does not have a replica
- * that matches the target block.  
+ * that matches the target block.
  */
 public class ReplicaNotFoundException extends IOException {
   private static final long serialVersionUID = 1L;
-  public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
-  public final static String UNFINALIZED_REPLICA = 
-    "Cannot append to an unfinalized replica ";
-  public final static String UNFINALIZED_AND_NONRBW_REPLICA = 
-    "Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
+  public final static String NON_RBW_REPLICA =
+      "Cannot recover a non-RBW replica ";
+  public final static String UNFINALIZED_REPLICA =
+      "Cannot append to an unfinalized replica ";
+  public final static String UNFINALIZED_AND_NONRBW_REPLICA =
+      "Cannot recover append/close to a replica that's not FINALIZED and not RBW"
+          + " ";
   public final static String NON_EXISTENT_REPLICA =
-    "Cannot append to a non-existent replica ";
+      "Cannot append to a non-existent replica ";
   public final static String UNEXPECTED_GS_REPLICA =
-    "Cannot append to a replica with unexpected generation stamp ";
+      "Cannot append to a replica with unexpected generation stamp ";
 
   public ReplicaNotFoundException() {
     super();
@@ -46,7 +48,7 @@ public class ReplicaNotFoundException extends IOException {
   public ReplicaNotFoundException(ExtendedBlock b) {
     super("Replica not found for " + b);
   }
-  
+
   public ReplicaNotFoundException(String msg) {
     super(msg);
   }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java

@@ -25,8 +25,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 
 /**
  * This exception is thrown when the name node is in safe mode.
- * Client cannot modified namespace until the safe mode is off. 
- * 
+ * Client cannot modified namespace until the safe mode is off.
+ *
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -35,4 +35,4 @@ public class SafeModeException extends IOException {
   public SafeModeException(String msg) {
     super(msg);
   }
-}
+}

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/AbstractNNFailoverProxyProvider.java

@@ -20,22 +20,21 @@ package org.apache.hadoop.hdfs.server.namenode.ha;
 
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.retry.FailoverProxyProvider;
 
 public abstract class AbstractNNFailoverProxyProvider<T> implements
-   FailoverProxyProvider <T> {
+    FailoverProxyProvider <T> {
 
   private AtomicBoolean fallbackToSimpleAuth;
 
   /**
    * Inquire whether logical HA URI is used for the implementation. If it is
-   * used, a special token handling may be needed to make sure a token acquired 
-   * from a node in the HA pair can be used against the other node. 
+   * used, a special token handling may be needed to make sure a token acquired
+   * from a node in the HA pair can be used against the other node.
    *
    * @return true if logical HA URI is used. false, if not used.
    */
-  public abstract boolean useLogicalURI(); 
+  public abstract boolean useLogicalURI();
 
   /**
    * Set for tracking if a secure client falls back to simple auth.  This method

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/WrappedFailoverProxyProvider.java

@@ -30,14 +30,14 @@ import org.apache.hadoop.io.retry.FailoverProxyProvider;
 public class WrappedFailoverProxyProvider<T> extends
     AbstractNNFailoverProxyProvider<T> {
   private final FailoverProxyProvider<T> proxyProvider;
-  
+
   /**
    * Wrap the given instance of an old FailoverProxyProvider.
    */
   public WrappedFailoverProxyProvider(FailoverProxyProvider<T> provider) {
     proxyProvider = provider;
   }
-    
+
   @Override
   public Class<T> getInterface() {
     return proxyProvider.getInterface();

+ 14 - 13
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java

@@ -28,22 +28,24 @@ public class DatanodeStorage {
   /** The state of the storage. */
   public enum State {
     NORMAL,
-    
+
     /**
-     * A storage that represents a read-only path to replicas stored on a shared storage device.
-     * Replicas on {@link #READ_ONLY_SHARED} storage are not counted towards live replicas.
-     * 
+     * A storage that represents a read-only path to replicas stored on a shared
+     * storage device. Replicas on {@link #READ_ONLY_SHARED} storage are not
+     * counted towards live replicas.
+     *
      * <p>
-     * In certain implementations, a {@link #READ_ONLY_SHARED} storage may be correlated to 
-     * its {@link #NORMAL} counterpart using the {@link DatanodeStorage#storageID}.  This
-     * property should be used for debugging purposes only.
-     * </p> 
+     * In certain implementations, a {@link #READ_ONLY_SHARED} storage may be
+     * correlated to its {@link #NORMAL} counterpart using the
+     * {@link DatanodeStorage#storageID}.  This property should be used for
+     * debugging purposes only.
+     * </p>
      */
     READ_ONLY_SHARED,
 
-    FAILED;
+    FAILED
   }
-  
+
   private final String storageID;
   private final State state;
   private final StorageType storageType;
@@ -91,10 +93,9 @@ public class DatanodeStorage {
     try {
       // Attempt to parse the UUID.
       if (storageID != null && storageID.indexOf(STORAGE_ID_PREFIX) == 0) {
-        UUID.fromString(storageID.substring(STORAGE_ID_PREFIX.length()));
         return true;
       }
-    } catch (IllegalArgumentException iae) {
+    } catch (IllegalArgumentException ignored) {
     }
 
     return false;
@@ -104,7 +105,7 @@ public class DatanodeStorage {
   public String toString() {
     return "DatanodeStorage["+ storageID + "," + storageType + "," + state +"]";
   }
-  
+
   @Override
   public boolean equals(Object other){
     if (other == this) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorageReport.java

@@ -39,4 +39,4 @@ public class DatanodeStorageReport {
   public StorageReport[] getStorageReports() {
     return storageReports;
   }
-}
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java

@@ -29,7 +29,7 @@ public class StorageReport {
   private final long blockPoolUsed;
 
   public static final StorageReport[] EMPTY_ARRAY = {};
-  
+
   public StorageReport(DatanodeStorage storage, boolean failed,
       long capacity, long dfsUsed, long remaining, long bpUsed) {
     this.storage = storage;

Some files were not shown because too many files changed in this diff