Browse Source

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

Haohui Mai 9 years ago
parent
commit
7136e8c558
100 changed files with 2137 additions and 2357 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. 3 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. 193 251
      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. 71 93
      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. 2 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
  18. 2 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
  19. 6 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
  20. 54 60
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
  21. 142 185
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  22. 2 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExtendedBlockId.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java
  24. 10 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/KeyProviderCache.java
  25. 1 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/NameNodeProxiesClient.java
  26. 18 21
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/PeerCache.java
  27. 73 75
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java
  28. 62 64
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
  30. 0 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
  31. 19 20
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
  32. 30 19
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
  33. 5 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
  34. 16 15
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
  35. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/CorruptFileBlockIterator.java
  36. 4 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java
  37. 10 10
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/package-info.java
  39. 27 22
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/Event.java
  40. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/BasicInetPeer.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/DomainPeer.java
  42. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/EncryptedPeer.java
  43. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/NioInetPeer.java
  44. 18 18
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/net/Peer.java
  45. 13 18
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/Block.java
  46. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockLocalPathInfo.java
  47. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
  48. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveEntry.java
  49. 10 11
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveInfo.java
  50. 4 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
  51. 11 15
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveStats.java
  52. 10 12
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
  53. 1 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
  54. 6 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolStats.java
  55. 22 22
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  56. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  57. 4 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DSQuotaExceededException.java
  58. 4 8
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
  59. 47 30
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
  60. 1 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfoWithStorage.java
  61. 3 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeLocalInfo.java
  62. 2 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
  63. 6 13
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java
  64. 3 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java
  65. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java
  66. 2 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LastBlockWithStatus.java
  67. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  68. 23 26
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java
  69. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
  70. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/NSQuotaExceededException.java
  71. 5 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
  72. 28 27
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotDiffReport.java
  73. 37 37
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java
  74. 4 12
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
  75. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/BlockConstructionStage.java
  76. 9 9
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtoUtil.java
  77. 28 28
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  78. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/IOStreamPair.java
  79. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Op.java
  80. 20 22
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketHeader.java
  81. 21 21
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PacketReceiver.java
  82. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/PipelineAck.java
  83. 20 30
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
  84. 61 58
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Sender.java
  85. 4 5
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/TrustedChannelResolver.java
  86. 61 58
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/DataTransferSaslUtil.java
  87. 52 56
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslDataTransferClient.java
  88. 6 6
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslParticipant.java
  89. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/sasl/SaslResponseWithNegotiatedCipherOption.java
  90. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolPB.java
  91. 20 16
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientDatanodeProtocolTranslatorPB.java
  92. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolPB.java
  93. 150 196
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  94. 334 330
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  95. 11 9
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenIdentifier.java
  96. 4 4
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java
  97. 17 15
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  98. 2 2
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/CachingStrategy.java
  99. 11 9
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
  100. 3 3
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.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);
   FORCE((short) 0x01);
   private final short mode;
   private final short mode;
 
 
-  private CacheFlag(short mode) {
+  CacheFlag(short mode) {
     this.mode = 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;
 package org.apache.hadoop.fs;
 
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -33,14 +31,13 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 public class HdfsBlockLocation extends BlockLocation {
 public class HdfsBlockLocation extends BlockLocation {
 
 
   private final LocatedBlock block;
   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
     // Initialize with data from passed in BlockLocation
     super(loc);
     super(loc);
     this.block = block;
     this.block = block;
   }
   }
-  
+
   public LocatedBlock getLocatedBlock() {
   public LocatedBlock getLocatedBlock() {
     return block;
     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
 @InterfaceAudience.Private
 public class XAttr {
 public class XAttr {
 
 
-  public static enum NameSpace {
+  public enum NameSpace {
     USER,
     USER,
     TRUSTED,
     TRUSTED,
     SECURITY,
     SECURITY,
     SYSTEM,
     SYSTEM,
-    RAW;
+    RAW
   }
   }
 
 
   private final NameSpace ns;
   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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 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
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
@@ -41,7 +41,8 @@ public class BlockMissingException extends IOException {
    * @param filename name of corrupted file
    * @param filename name of corrupted file
    * @param description a description of the corruption details
    * @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);
     super(description);
     this.filename = filename;
     this.filename = filename;
     this.offset = offset;
     this.offset = offset;

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

@@ -33,7 +33,7 @@ import org.apache.hadoop.util.DataChecksum;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public interface BlockReader extends ByteBufferReadable, Closeable {
 public interface BlockReader extends ByteBufferReadable, Closeable {
-  
+
 
 
   /* same interface as inputStream java.io.InputStream#read()
   /* same interface as inputStream java.io.InputStream#read()
    * used by DFSInputStream#read()
    * used by DFSInputStream#read()
@@ -57,7 +57,7 @@ public interface BlockReader extends ByteBufferReadable, Closeable {
    * network I/O.
    * network I/O.
    * This may return more than what is actually present in the block.
    * This may return more than what is actually present in the block.
    */
    */
-  int available() throws IOException;
+  int available();
 
 
   /**
   /**
    * Close the block reader.
    * Close the block reader.
@@ -87,7 +87,7 @@ public interface BlockReader extends ByteBufferReadable, Closeable {
    * @return              true only if this is a local read.
    * @return              true only if this is a local read.
    */
    */
   boolean isLocal();
   boolean isLocal();
-  
+
   /**
   /**
    * @return              true only if this is a short-circuit read.
    * @return              true only if this is a short-circuit read.
    *                      All short-circuit reads are also local.
    *                      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;
 import org.slf4j.LoggerFactory;
 
 
 
 
-/** 
+/**
  * Utility class to create BlockReader implementations.
  * Utility class to create BlockReader implementations.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
@@ -127,7 +127,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   /**
   /**
    * The name of this client.
    * The name of this client.
    */
    */
-  private String clientName; 
+  private String clientName;
 
 
   /**
   /**
    * The DataNode we're talking to.
    * The DataNode we're talking to.
@@ -170,7 +170,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
   private RemotePeerFactory remotePeerFactory;
   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;
   private UserGroupInformation userGroupInformation;
 
 
@@ -313,7 +314,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * There are a few caches that are important here.
    * There are a few caches that are important here.
    *
    *
    * The ShortCircuitCache stores file descriptor objects which have been passed
    * 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
    * 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
    * 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;
       return null;
     }
     }
     if (clientContext.getDisableLegacyBlockReaderLocal()) {
     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;
       return null;
     }
     }
     IOException ioe;
     IOException ioe;
@@ -470,7 +471,8 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
       return null;
       return null;
     }
     }
     ShortCircuitCache cache = clientContext.getShortCircuitCache();
     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);
     ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
     InvalidToken exc = info.getInvalidTokenException();
     InvalidToken exc = info.getInvalidTokenException();
     if (exc != null) {
     if (exc != null) {
@@ -501,14 +503,15 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    *
    *
    * @return    Null if we could not communicate with the datanode,
    * @return    Null if we could not communicate with the datanode,
    *            a new ShortCircuitReplicaInfo object otherwise.
    *            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
   @Override
   public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
   public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
     if (createShortCircuitReplicaInfoCallback != null) {
     if (createShortCircuitReplicaInfoCallback != null) {
       ShortCircuitReplicaInfo info =
       ShortCircuitReplicaInfo info =
-        createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
+          createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
       if (info != null) return info;
       if (info != null) return info;
     }
     }
     LOG.trace("{}: trying to create ShortCircuitReplicaInfo.", this);
     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.
           // 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
           // We temporarily disable the domain socket path for a few minutes in
           // this case, to prevent wasting more time on it.
           // 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);
               "Disabling domain socket " + peer.getDomainSocket(), e);
           IOUtilsClient.cleanup(LOG, peer);
           IOUtilsClient.cleanup(LOG, peer);
           clientContext.getDomainSocketFactory()
           clientContext.getDomainSocketFactory()
@@ -564,11 +567,11 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
    * Request file descriptors from a DomainPeer.
    * Request file descriptors from a DomainPeer.
    *
    *
    * @param peer   The peer to use for communication.
    * @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.
    *               new ShortCircuitReplica.
-   * 
+   *
    * @return  A ShortCircuitReplica object if we could communicate with the
    * @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
    * @throws  IOException If we encountered an I/O exception while communicating
    *          with the datanode.
    *          with the datanode.
    */
    */
@@ -682,7 +685,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
         }
         }
         if (curPeer.fromCache) {
         if (curPeer.fromCache) {
           // Handle an I/O error we got when using a cached peer.  These are
           // 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);
           LOG.debug("Closed potentially stale domain peer {}", peer, ioe);
         } else {
         } else {
           // Handle an I/O error we got when using a newly created domain peer.
           // 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 {
   public static class BlockReaderPeer {
     final Peer peer;
     final Peer peer;
     final boolean fromCache;
     final boolean fromCache;
-    
+
     BlockReaderPeer(Peer peer, boolean fromCache) {
     BlockReaderPeer(Peer peer, boolean fromCache) {
       this.peer = peer;
       this.peer = peer;
       this.fromCache = fromCache;
       this.fromCache = fromCache;
@@ -800,7 +803,7 @@ public class BlockReaderFactory implements ShortCircuitReplicaCreator {
     }
     }
     try {
     try {
       Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
       Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
-        datanode);
+          datanode);
       LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
       LOG.trace("nextTcpPeer: created newConnectedPeer {}", peer);
       return new BlockReaderPeer(peer, false);
       return new BlockReaderPeer(peer, false);
     } catch (IOException e) {
     } 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>
  * <ul>
  * <li>The client performing short circuit reads must be configured at the
  * <li>The client performing short circuit reads must be configured at the
  * datanode.</li>
  * 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
  * file for the block using
  * {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
  * {@link org.apache.hadoop.hdfs.server.datanode.DataXceiver#requestShortCircuitFds}.
  * </li>
  * </li>
@@ -155,7 +155,7 @@ class BlockReaderLocal implements BlockReader {
    * The Checksum FileChannel.
    * The Checksum FileChannel.
    */
    */
   private final FileChannel checksumIn;
   private final FileChannel checksumIn;
-  
+
   /**
   /**
    * Checksum type and size.
    * Checksum type and size.
    */
    */
@@ -170,12 +170,12 @@ class BlockReaderLocal implements BlockReader {
    * Name of the block, for logging purposes.
    * Name of the block, for logging purposes.
    */
    */
   private final String filename;
   private final String filename;
-  
+
   /**
   /**
    * Block ID and Block Pool ID.
    * Block ID and Block Pool ID.
    */
    */
   private final ExtendedBlock block;
   private final ExtendedBlock block;
-  
+
   /**
   /**
    * Cache of Checksum#bytesPerChecksum.
    * Cache of Checksum#bytesPerChecksum.
    */
    */
@@ -204,11 +204,11 @@ class BlockReaderLocal implements BlockReader {
    * size of a single chunk, even if {@link #zeroReadaheadRequested} is true.
    * 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
    * The reason is because we need to do a certain amount of buffering in order
    * to do checksumming.
    * to do checksumming.
-   * 
+   *
    * This determines how many bytes we'll use out of dataBuf and checksumBuf.
    * 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?
    * 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
    * 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;
   private final int maxReadaheadLength;
 
 
@@ -335,9 +335,8 @@ class BlockReaderLocal implements BlockReader {
    */
    */
   private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
   private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
       throws IOException {
       throws IOException {
-    TraceScope scope = tracer.newScope(
-        "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")");
-    try {
+    try (TraceScope ignored = tracer.newScope(
+        "BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
       int total = 0;
       int total = 0;
       long startDataPos = dataPos;
       long startDataPos = dataPos;
       int startBufPos = buf.position();
       int startBufPos = buf.position();
@@ -358,7 +357,8 @@ class BlockReaderLocal implements BlockReader {
           buf.limit(buf.position());
           buf.limit(buf.position());
           buf.position(startBufPos);
           buf.position(startBufPos);
           createChecksumBufIfNeeded();
           createChecksumBufIfNeeded();
-          int checksumsNeeded = (total + bytesPerChecksum - 1) / bytesPerChecksum;
+          int checksumsNeeded = (total + bytesPerChecksum - 1) /
+              bytesPerChecksum;
           checksumBuf.clear();
           checksumBuf.clear();
           checksumBuf.limit(checksumsNeeded * checksumSize);
           checksumBuf.limit(checksumsNeeded * checksumSize);
           long checksumPos = BlockMetadataHeader.getHeaderSize()
           long checksumPos = BlockMetadataHeader.getHeaderSize()
@@ -367,8 +367,8 @@ class BlockReaderLocal implements BlockReader {
             int nRead = checksumIn.read(checksumBuf, checksumPos);
             int nRead = checksumIn.read(checksumBuf, checksumPos);
             if (nRead < 0) {
             if (nRead < 0) {
               throw new IOException("Got unexpected checksum file EOF at " +
               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;
             checksumPos += nRead;
           }
           }
@@ -380,24 +380,16 @@ class BlockReaderLocal implements BlockReader {
         }
         }
       }
       }
       return total;
       return total;
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
   private boolean createNoChecksumContext() {
   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() {
   private void releaseNoChecksumContext() {
@@ -453,14 +445,14 @@ class BlockReaderLocal implements BlockReader {
   /**
   /**
    * Fill the data buffer.  If necessary, validate the data against the
    * Fill the data buffer.  If necessary, validate the data against the
    * checksums.
    * checksums.
-   * 
+   *
    * We always want the offsets of the data contained in dataBuf to be
    * We always want the offsets of the data contained in dataBuf to be
    * aligned to the chunk boundary.  If we are validating checksums, we
    * aligned to the chunk boundary.  If we are validating checksums, we
    * accomplish this by seeking backwards in the file until we're on a
    * accomplish this by seeking backwards in the file until we're on a
    * chunk boundary.  (This is necessary because we can't checksum a
    * chunk boundary.  (This is necessary because we can't checksum a
    * partial chunk.)  If we are not validating checksums, we simply only
    * partial chunk.)  If we are not validating checksums, we simply only
    * fill the latter part of dataBuf.
    * fill the latter part of dataBuf.
-   * 
+   *
    * @param canSkipChecksum  true if we can skip checksumming.
    * @param canSkipChecksum  true if we can skip checksumming.
    * @return                 true if we hit EOF.
    * @return                 true if we hit EOF.
    * @throws IOException
    * @throws IOException
@@ -473,11 +465,11 @@ class BlockReaderLocal implements BlockReader {
     dataBuf.limit(maxReadaheadLength);
     dataBuf.limit(maxReadaheadLength);
     if (canSkipChecksum) {
     if (canSkipChecksum) {
       dataBuf.position(slop);
       dataBuf.position(slop);
-      fillBuffer(dataBuf, canSkipChecksum);
+      fillBuffer(dataBuf, true);
     } else {
     } else {
       dataPos -= slop;
       dataPos -= slop;
       dataBuf.position(0);
       dataBuf.position(0);
-      fillBuffer(dataBuf, canSkipChecksum);
+      fillBuffer(dataBuf, false);
     }
     }
     dataBuf.limit(dataBuf.position());
     dataBuf.limit(dataBuf.position());
     dataBuf.position(Math.min(dataBuf.position(), slop));
     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
    * efficiency's sake. As described above, all non-checksum-chunk-aligned
    * reads will be served from the slower read path.
    * 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.
    * @param canSkipChecksum  True if we can skip checksums.
    */
    */
   private synchronized int readWithBounceBuffer(ByteBuffer buf,
   private synchronized int readWithBounceBuffer(ByteBuffer buf,
@@ -621,7 +613,7 @@ class BlockReaderLocal implements BlockReader {
   }
   }
 
 
   @Override
   @Override
-  public int available() throws IOException {
+  public int available() {
     // We never do network I/O in BlockReaderLocal.
     // We never do network I/O in BlockReaderLocal.
     return Integer.MAX_VALUE;
     return Integer.MAX_VALUE;
   }
   }
@@ -660,8 +652,8 @@ class BlockReaderLocal implements BlockReader {
 
 
   /**
   /**
    * Get or create a memory map for this replica.
    * 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
    * will always read pre-checksummed data, and one that may read data that
    * hasn't been checksummed.
    * hasn't been checksummed.
    *
    *
@@ -671,13 +663,13 @@ class BlockReaderLocal implements BlockReader {
    * If we fetch the latter, we don't bother with anchoring.
    * If we fetch the latter, we don't bother with anchoring.
    *
    *
    * @param opts     The options to use, such as SKIP_CHECKSUMS.
    * @param opts     The options to use, such as SKIP_CHECKSUMS.
-   * 
+   *
    * @return         null on failure; the ClientMmap otherwise.
    * @return         null on failure; the ClientMmap otherwise.
    */
    */
   @Override
   @Override
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
   public ClientMmap getClientMmap(EnumSet<ReadOption> opts) {
     boolean anchor = verifyChecksum &&
     boolean anchor = verifyChecksum &&
-        (opts.contains(ReadOption.SKIP_CHECKSUMS) == false);
+        !opts.contains(ReadOption.SKIP_CHECKSUMS);
     if (anchor) {
     if (anchor) {
       if (!createNoChecksumContext()) {
       if (!createNoChecksumContext()) {
         LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
         LOG.trace("can't get an mmap for {} of {} since SKIP_CHECKSUMS was not "
@@ -696,7 +688,7 @@ class BlockReaderLocal implements BlockReader {
     }
     }
     return clientMmap;
     return clientMmap;
   }
   }
-  
+
   @VisibleForTesting
   @VisibleForTesting
   boolean getVerifyChecksum() {
   boolean getVerifyChecksum() {
     return this.verifyChecksum;
     return this.verifyChecksum;
@@ -706,7 +698,7 @@ class BlockReaderLocal implements BlockReader {
   int getMaxReadaheadLength() {
   int getMaxReadaheadLength() {
     return this.maxReadaheadLength;
     return this.maxReadaheadLength;
   }
   }
-  
+
   /**
   /**
    * Make the replica anchorable.  Normally this can only be done by the
    * Make the replica anchorable.  Normally this can only be done by the
    * DataNode.  This method is only for testing.
    * 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;
 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
  * 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
  * permissions on the datanode to be set so that clients can directly access the
@@ -90,7 +90,8 @@ class BlockReaderLocalLegacy implements BlockReader {
     LocalDatanodeInfo() {
     LocalDatanodeInfo() {
       final int cacheSize = 10000;
       final int cacheSize = 10000;
       final float hashTableLoadFactor = 0.75f;
       final float hashTableLoadFactor = 0.75f;
-      int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor) + 1;
+      int hashTableCapacity = (int) Math.ceil(cacheSize / hashTableLoadFactor)
+          + 1;
       cache = Collections
       cache = Collections
           .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
           .synchronizedMap(new LinkedHashMap<ExtendedBlock, BlockLocalPathInfo>(
               hashTableCapacity, hashTableLoadFactor, true) {
               hashTableCapacity, hashTableLoadFactor, true) {
@@ -123,7 +124,7 @@ class BlockReaderLocalLegacy implements BlockReader {
       }
       }
       return proxy;
       return proxy;
     }
     }
-    
+
     private synchronized void resetDatanodeProxy() {
     private synchronized void resetDatanodeProxy() {
       if (null != proxy) {
       if (null != proxy) {
         RPC.stopProxy(proxy);
         RPC.stopProxy(proxy);
@@ -135,7 +136,8 @@ class BlockReaderLocalLegacy implements BlockReader {
       return cache.get(b);
       return cache.get(b);
     }
     }
 
 
-    private void setBlockLocalPathInfo(ExtendedBlock b, BlockLocalPathInfo info) {
+    private void setBlockLocalPathInfo(ExtendedBlock b,
+        BlockLocalPathInfo info) {
       cache.put(b, info);
       cache.put(b, info);
     }
     }
 
 
@@ -143,10 +145,11 @@ class BlockReaderLocalLegacy implements BlockReader {
       cache.remove(b);
       cache.remove(b);
     }
     }
   }
   }
-  
+
   // Multiple datanodes could be running on the local machine. Store proxies in
   // Multiple datanodes could be running on the local machine. Store proxies in
   // a map keyed by the ipc port of the datanode.
   // 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 dataIn; // reader for the data file
   private final FileInputStream checksumIn;   // reader for the checksum 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.
    * checksum read at construction to position the read cursor correctly.
    */
    */
   private int offsetFromChunkBoundary;
   private int offsetFromChunkBoundary;
-  
+
   private byte[] skipBuf = null;
   private byte[] skipBuf = null;
 
 
   /**
   /**
@@ -188,7 +191,7 @@ class BlockReaderLocalLegacy implements BlockReader {
   static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
   static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
       UserGroupInformation userGroupInformation,
       UserGroupInformation userGroupInformation,
       Configuration configuration, String file, ExtendedBlock blk,
       Configuration configuration, String file, ExtendedBlock blk,
-      Token<BlockTokenIdentifier> token, DatanodeInfo node, 
+      Token<BlockTokenIdentifier> token, DatanodeInfo node,
       long startOffset, long length, StorageType storageType,
       long startOffset, long length, StorageType storageType,
       Tracer tracer) throws IOException {
       Tracer tracer) throws IOException {
     final ShortCircuitConf scConf = conf.getShortCircuitConf();
     final ShortCircuitConf scConf = conf.getShortCircuitConf();
@@ -234,12 +237,12 @@ class BlockReaderLocalLegacy implements BlockReader {
             new DataInputStream(checksumIn), blk);
             new DataInputStream(checksumIn), blk);
         long firstChunkOffset = startOffset
         long firstChunkOffset = startOffset
             - (startOffset % checksum.getBytesPerChecksum());
             - (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 {
       } 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) {
     } catch (IOException e) {
       // remove from cache
       // remove from cache
@@ -260,7 +263,7 @@ class BlockReaderLocalLegacy implements BlockReader {
     }
     }
     return localBlockReader;
     return localBlockReader;
   }
   }
-  
+
   private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
   private static synchronized LocalDatanodeInfo getLocalDatanodeInfo(int port) {
     LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
     LocalDatanodeInfo ldInfo = localDatanodeInfoMap.get(port);
     if (ldInfo == null) {
     if (ldInfo == null) {
@@ -269,19 +272,20 @@ class BlockReaderLocalLegacy implements BlockReader {
     }
     }
     return ldInfo;
     return ldInfo;
   }
   }
-  
+
   private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
   private static BlockLocalPathInfo getBlockPathInfo(UserGroupInformation ugi,
       ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
       ExtendedBlock blk, DatanodeInfo node, Configuration conf, int timeout,
       Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname,
       Token<BlockTokenIdentifier> token, boolean connectToDnViaHostname,
       StorageType storageType) throws IOException {
       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,
     ClientDatanodeProtocol proxy = localDatanodeInfo.getDatanodeProxy(ugi, node,
         conf, timeout, connectToDnViaHostname);
         conf, timeout, connectToDnViaHostname);
     try {
     try {
       // make RPC to local datanode to find local pathnames of blocks
       // make RPC to local datanode to find local pathnames of blocks
       pathinfo = proxy.getBlockLocalPathInfo(blk, token);
       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,
       // 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
       // our next attempt to read from the cached path would fail to find the
       // file.  Additionally, the failure would cause us to disable legacy
       // file.  Additionally, the failure would cause us to disable legacy
@@ -299,7 +303,7 @@ class BlockReaderLocalLegacy implements BlockReader {
     }
     }
     return pathinfo;
     return pathinfo;
   }
   }
-  
+
   private static int getSlowReadBufferNumChunks(int bufferSizeBytes,
   private static int getSlowReadBufferNumChunks(int bufferSizeBytes,
       int bytesPerChecksum) {
       int bytesPerChecksum) {
     if (bufferSizeBytes < bytesPerChecksum) {
     if (bufferSizeBytes < bytesPerChecksum) {
@@ -315,17 +319,15 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
   }
 
 
   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
   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 {
       Tracer tracer) throws IOException {
-    this(conf, hdfsfile, block, token, startOffset, length, pathinfo,
+    this(conf, hdfsfile, block, startOffset,
         DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
         DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
         dataIn, startOffset, null, tracer);
         dataIn, startOffset, null, tracer);
   }
   }
 
 
   private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
   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,
       boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
       FileInputStream checksumIn, Tracer tracer) throws IOException {
       FileInputStream checksumIn, Tracer tracer) throws IOException {
     this.filename = hdfsfile;
     this.filename = hdfsfile;
@@ -343,17 +345,20 @@ class BlockReaderLocalLegacy implements BlockReader {
 
 
     final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
     final int chunksPerChecksumRead = getSlowReadBufferNumChunks(
         conf.getShortCircuitBufferSize(), bytesPerChecksum);
         conf.getShortCircuitBufferSize(), bytesPerChecksum);
-    slowReadBuff = bufferPool.getBuffer(bytesPerChecksum * chunksPerChecksumRead);
+    slowReadBuff = bufferPool.getBuffer(
+        bytesPerChecksum * chunksPerChecksumRead);
     checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
     checksumBuff = bufferPool.getBuffer(checksumSize * chunksPerChecksumRead);
     // Initially the buffers have nothing to read.
     // Initially the buffers have nothing to read.
     slowReadBuff.flip();
     slowReadBuff.flip();
     checksumBuff.flip();
     checksumBuff.flip();
     boolean success = false;
     boolean success = false;
     try {
     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);
       IOUtils.skipFully(dataIn, firstChunkOffset);
       if (checksumIn != null) {
       if (checksumIn != null) {
-        long checkSumOffset = (firstChunkOffset / bytesPerChecksum) * checksumSize;
+        long checkSumOffset = (firstChunkOffset / bytesPerChecksum) *
+            checksumSize;
         IOUtils.skipFully(checksumIn, checkSumOffset);
         IOUtils.skipFully(checksumIn, checkSumOffset);
       }
       }
       success = true;
       success = true;
@@ -371,9 +376,8 @@ class BlockReaderLocalLegacy implements BlockReader {
    */
    */
   private int fillBuffer(FileInputStream stream, ByteBuffer buf)
   private int fillBuffer(FileInputStream stream, ByteBuffer buf)
       throws IOException {
       throws IOException {
-    TraceScope scope = tracer.
-        newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")");
-    try {
+    try (TraceScope ignored = tracer.
+        newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
       int bytesRead = stream.getChannel().read(buf);
       int bytesRead = stream.getChannel().read(buf);
       if (bytesRead < 0) {
       if (bytesRead < 0) {
         //EOF
         //EOF
@@ -388,11 +392,9 @@ class BlockReaderLocalLegacy implements BlockReader {
         bytesRead += n;
         bytesRead += n;
       }
       }
       return bytesRead;
       return bytesRead;
-    } finally {
-      scope.close();
     }
     }
   }
   }
-  
+
   /**
   /**
    * Utility method used by read(ByteBuffer) to partially copy a ByteBuffer into
    * Utility method used by read(ByteBuffer) to partially copy a ByteBuffer into
    * another.
    * another.
@@ -426,7 +428,8 @@ class BlockReaderLocalLegacy implements BlockReader {
       if (slowReadBuff.hasRemaining()) {
       if (slowReadBuff.hasRemaining()) {
         // There are remaining bytes from a small read available. This usually
         // There are remaining bytes from a small read available. This usually
         // means this read is unaligned, which falls back to the slow path.
         // 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);
         writeSlice(slowReadBuff, buf, fromSlowReadBuff);
         nRead += fromSlowReadBuff;
         nRead += fromSlowReadBuff;
       }
       }
@@ -458,8 +461,10 @@ class BlockReaderLocalLegacy implements BlockReader {
 
 
       // offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
       // offsetFromChunkBoundary > 0 => unaligned read, use slow path to read
       // until chunk boundary
       // 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);
         int readResult = fillSlowReadBuffer(toRead);
         if (readResult == -1) {
         if (readResult == -1) {
           return nRead;
           return nRead;
@@ -470,7 +475,8 @@ class BlockReaderLocalLegacy implements BlockReader {
         }
         }
       }
       }
     } else {
     } 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);
       nRead = doByteBufferRead(buf);
       if (nRead > 0) {
       if (nRead > 0) {
         buf.position(buf.position() + nRead);
         buf.position(buf.position() + nRead);
@@ -512,7 +518,7 @@ class BlockReaderLocalLegacy implements BlockReader {
     if (verifyChecksum) {
     if (verifyChecksum) {
       assert buf.remaining() % bytesPerChecksum == 0;
       assert buf.remaining() % bytesPerChecksum == 0;
     }
     }
-    int dataRead = -1;
+    int dataRead;
 
 
     int oldpos = buf.position();
     int oldpos = buf.position();
     // Read as much as we can into the buffer.
     // Read as much as we can into the buffer.
@@ -528,9 +534,10 @@ class BlockReaderLocalLegacy implements BlockReader {
       toChecksum.limit(oldpos + dataRead);
       toChecksum.limit(oldpos + dataRead);
 
 
       checksumBuff.clear();
       checksumBuff.clear();
-      // Equivalent to (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
+      // Equivalent to
+      // (int)Math.ceil(toChecksum.remaining() * 1.0 / bytesPerChecksum );
       int numChunks =
       int numChunks =
-        (toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
+          (toChecksum.remaining() + bytesPerChecksum - 1) / bytesPerChecksum;
       checksumBuff.limit(checksumSize * numChunks);
       checksumBuff.limit(checksumSize * numChunks);
 
 
       fillBuffer(checksumIn, checksumBuff);
       fillBuffer(checksumIn, checksumBuff);
@@ -571,7 +578,7 @@ class BlockReaderLocalLegacy implements BlockReader {
    * @return the number of bytes available to read, or -1 if EOF.
    * @return the number of bytes available to read, or -1 if EOF.
    */
    */
   private synchronized int fillSlowReadBuffer(int len) throws IOException {
   private synchronized int fillSlowReadBuffer(int len) throws IOException {
-    int nRead = -1;
+    int nRead;
     if (slowReadBuff.hasRemaining()) {
     if (slowReadBuff.hasRemaining()) {
       // Already got data, good to go.
       // Already got data, good to go.
       nRead = Math.min(len, slowReadBuff.remaining());
       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
       // 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
       // next chunk boundary, since we try and read in multiples of a chunk
       int nextChunk = len + offsetFromChunkBoundary +
       int nextChunk = len + offsetFromChunkBoundary +
-          (bytesPerChecksum - ((len + offsetFromChunkBoundary) % bytesPerChecksum));
+          (bytesPerChecksum -
+              ((len + offsetFromChunkBoundary) % bytesPerChecksum));
       int limit = Math.min(nextChunk, slowReadBuff.capacity());
       int limit = Math.min(nextChunk, slowReadBuff.capacity());
       assert limit % bytesPerChecksum == 0;
       assert limit % bytesPerChecksum == 0;
 
 
@@ -598,7 +606,8 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
   }
 
 
   @Override
   @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);
     LOG.trace("read off {} len {}", off, len);
     if (!verifyChecksum) {
     if (!verifyChecksum) {
       return dataIn.read(buf, off, len);
       return dataIn.read(buf, off, len);
@@ -625,19 +634,19 @@ class BlockReaderLocalLegacy implements BlockReader {
     if (!verifyChecksum) {
     if (!verifyChecksum) {
       return dataIn.skip(n);
       return dataIn.skip(n);
     }
     }
-  
+
     // caller made sure newPosition is not beyond EOF.
     // caller made sure newPosition is not beyond EOF.
     int remaining = slowReadBuff.remaining();
     int remaining = slowReadBuff.remaining();
     int position = slowReadBuff.position();
     int position = slowReadBuff.position();
     int newPosition = position + (int)n;
     int newPosition = position + (int)n;
-  
+
     // if the new offset is already read into dataBuff, just reposition
     // if the new offset is already read into dataBuff, just reposition
     if (n <= remaining) {
     if (n <= remaining) {
       assert offsetFromChunkBoundary == 0;
       assert offsetFromChunkBoundary == 0;
       slowReadBuff.position(newPosition);
       slowReadBuff.position(newPosition);
       return n;
       return n;
     }
     }
-  
+
     // for small gap, read through to keep the data/checksum in sync
     // for small gap, read through to keep the data/checksum in sync
     if (n - remaining <= bytesPerChecksum) {
     if (n - remaining <= bytesPerChecksum) {
       slowReadBuff.position(position + remaining);
       slowReadBuff.position(position + remaining);
@@ -647,11 +656,11 @@ class BlockReaderLocalLegacy implements BlockReader {
       int ret = read(skipBuf, 0, (int)(n - remaining));
       int ret = read(skipBuf, 0, (int)(n - remaining));
       return (remaining + ret);
       return (remaining + ret);
     }
     }
-  
+
     // optimize for big gap: discard the current buffer, skip to
     // optimize for big gap: discard the current buffer, skip to
     // the beginning of the appropriate checksum chunk and then
     // the beginning of the appropriate checksum chunk and then
     // read to the middle of that chunk to be in sync with checksums.
     // 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
     // We can't use this.offsetFromChunkBoundary because we need to know how
     // many bytes of the offset were really read. Calling read(..) with a
     // many bytes of the offset were really read. Calling read(..) with a
     // positive this.offsetFromChunkBoundary causes that many bytes to get
     // positive this.offsetFromChunkBoundary causes that many bytes to get
@@ -661,7 +670,7 @@ class BlockReaderLocalLegacy implements BlockReader {
 
 
     slowReadBuff.position(slowReadBuff.limit());
     slowReadBuff.position(slowReadBuff.limit());
     checksumBuff.position(checksumBuff.limit());
     checksumBuff.position(checksumBuff.limit());
-  
+
     IOUtils.skipFully(dataIn, toskip);
     IOUtils.skipFully(dataIn, toskip);
     long checkSumOffset = (toskip / bytesPerChecksum) * checksumSize;
     long checkSumOffset = (toskip / bytesPerChecksum) * checksumSize;
     IOUtils.skipFully(checksumIn, checkSumOffset);
     IOUtils.skipFully(checksumIn, checkSumOffset);
@@ -708,7 +717,7 @@ class BlockReaderLocalLegacy implements BlockReader {
   }
   }
 
 
   @Override
   @Override
-  public int available() throws IOException {
+  public int available() {
     // We never do network I/O in BlockReaderLocalLegacy.
     // We never do network I/O in BlockReaderLocalLegacy.
     return Integer.MAX_VALUE;
     return Integer.MAX_VALUE;
   }
   }
@@ -717,7 +726,7 @@ class BlockReaderLocalLegacy implements BlockReader {
   public boolean isLocal() {
   public boolean isLocal() {
     return true;
     return true;
   }
   }
-  
+
   @Override
   @Override
   public boolean isShortCircuit() {
   public boolean isShortCircuit() {
     return true;
     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;
       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.
  * ClientContext contains context information for a client.
- * 
+ *
  * This allows us to share caches such as the socket cache across
  * This allows us to share caches such as the socket cache across
  * DFSClient instances.
  * DFSClient instances.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class ClientContext {
 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.
    * 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.
    * Name of context.
@@ -93,7 +93,7 @@ public class ClientContext {
   private volatile boolean disableLegacyBlockReaderLocal = false;
   private volatile boolean disableLegacyBlockReaderLocal = false;
 
 
   /** Creating byte[] for {@link DFSOutputStream}. */
   /** 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
    * Whether or not we complained about a DFSClient fetching a CacheContext that
@@ -152,7 +152,7 @@ public class ClientContext {
       if (!printedConfWarning) {
       if (!printedConfWarning) {
         printedConfWarning = true;
         printedConfWarning = true;
         LOG.warn("Existing client context '" + name + "' does not match " +
         LOG.warn("Existing client context '" + name + "' does not match " +
-            "requested configuration.  Existing: " + existing + 
+            "requested configuration.  Existing: " + existing +
             ", Requested: " + requested);
             ", Requested: " + requested);
       }
       }
     }
     }

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

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

@@ -54,11 +54,9 @@ import org.apache.hadoop.fs.CanUnbuffer;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
 import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.ReadOption;
 import org.apache.hadoop.fs.StorageType;
 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.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -84,13 +82,15 @@ import org.apache.htrace.core.Tracer;
 
 
 import com.google.common.annotations.VisibleForTesting;
 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.
  * negotiation of the namenode and various datanodes as necessary.
  ****************************************************************/
  ****************************************************************/
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class DFSInputStream extends FSInputStream
 public class DFSInputStream extends FSInputStream
-implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
+    implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     HasEnhancedByteBufferAccess, CanUnbuffer {
     HasEnhancedByteBufferAccess, CanUnbuffer {
   @VisibleForTesting
   @VisibleForTesting
   public static boolean tcpReadsDisabledForTesting = false;
   public static boolean tcpReadsDisabledForTesting = false;
@@ -127,7 +127,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
   /**
   /**
    * Track the ByteBuffers that we have handed out to readers.
    * Track the ByteBuffers that we have handed out to readers.
-   * 
+   *
    * The value type can be either ByteBufferPool or ClientMmap, depending on
    * The value type can be either ByteBufferPool or ClientMmap, depending on
    * whether we this is a memory-mapped buffer or not.
    * whether we this is a memory-mapped buffer or not.
    */
    */
@@ -136,7 +136,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   private synchronized IdentityHashStore<ByteBuffer, Object>
   private synchronized IdentityHashStore<ByteBuffer, Object>
         getExtendedReadBuffers() {
         getExtendedReadBuffers() {
     if (extendedReadBuffers == null) {
     if (extendedReadBuffers == null) {
-      extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
+      extendedReadBuffers = new IdentityHashStore<>(0);
     }
     }
     return extendedReadBuffers;
     return extendedReadBuffers;
   }
   }
@@ -176,7 +176,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     public long getTotalShortCircuitBytesRead() {
     public long getTotalShortCircuitBytesRead() {
       return totalShortCircuitBytesRead;
       return totalShortCircuitBytesRead;
     }
     }
-    
+
     /**
     /**
      * @return The total number of zero-copy bytes read.
      * @return The total number of zero-copy bytes read.
      */
      */
@@ -190,7 +190,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     public long getRemoteBytesRead() {
     public long getRemoteBytesRead() {
       return totalBytesRead - totalLocalBytesRead;
       return totalBytesRead - totalLocalBytesRead;
     }
     }
-    
+
     void addRemoteBytes(long amt) {
     void addRemoteBytes(long amt) {
       this.totalBytesRead += amt;
       this.totalBytesRead += amt;
     }
     }
@@ -219,7 +219,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       this.totalShortCircuitBytesRead = 0;
       this.totalShortCircuitBytesRead = 0;
       this.totalZeroCopyBytesRead = 0;
       this.totalZeroCopyBytesRead = 0;
     }
     }
-    
+
     private long totalBytesRead;
     private long totalBytesRead;
 
 
     private long totalLocalBytesRead;
     private long totalLocalBytesRead;
@@ -228,7 +228,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
     private long totalZeroCopyBytesRead;
     private long totalZeroCopyBytesRead;
   }
   }
-  
+
   /**
   /**
    * This variable tracks the number of failures since the start of the
    * 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
    * most recent user-facing operation. That is to say, it should be reset
@@ -242,19 +242,19 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    */
    */
   protected int failures = 0;
   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 */
    * parallel accesses to DFSInputStream (through ptreads) properly */
   private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
   private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
-             new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
+             new ConcurrentHashMap<>();
 
 
   private byte[] oneByteBuf; // used for 'int read()'
   private byte[] oneByteBuf; // used for 'int read()'
 
 
   void addToDeadNodes(DatanodeInfo dnInfo) {
   void addToDeadNodes(DatanodeInfo dnInfo) {
     deadNodes.put(dnInfo, dnInfo);
     deadNodes.put(dnInfo, dnInfo);
   }
   }
-  
+
   DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
   DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
-      LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
+      LocatedBlocks locatedBlocks) throws IOException {
     this.dfsClient = dfsClient;
     this.dfsClient = dfsClient;
     this.verifyChecksum = verifyChecksum;
     this.verifyChecksum = verifyChecksum;
     this.src = src;
     this.src = src;
@@ -269,8 +269,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Grab the open-file info from namenode
    * Grab the open-file info from namenode
    * @param refreshLocatedBlocks whether to re-fetch locatedblocks
    * @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();
     final DfsClientConf conf = dfsClient.getConf();
     synchronized(infoLock) {
     synchronized(infoLock) {
       lastBlockBeingWrittenLength =
       lastBlockBeingWrittenLength =
@@ -343,7 +342,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
         }
         final long len = readBlockLength(last);
         final long len = readBlockLength(last);
         last.getBlock().setNumBytes(len);
         last.getBlock().setNumBytes(len);
-        lastBlockBeingWrittenLength = len; 
+        lastBlockBeingWrittenLength = len;
       }
       }
     }
     }
 
 
@@ -356,30 +355,30 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   private long readBlockLength(LocatedBlock locatedblock) throws IOException {
   private long readBlockLength(LocatedBlock locatedblock) throws IOException {
     assert locatedblock != null : "LocatedBlock cannot be null";
     assert locatedblock != null : "LocatedBlock cannot be null";
     int replicaNotFoundCount = locatedblock.getLocations().length;
     int replicaNotFoundCount = locatedblock.getLocations().length;
-    
+
     final DfsClientConf conf = dfsClient.getConf();
     final DfsClientConf conf = dfsClient.getConf();
     for(DatanodeInfo datanode : locatedblock.getLocations()) {
     for(DatanodeInfo datanode : locatedblock.getLocations()) {
       ClientDatanodeProtocol cdp = null;
       ClientDatanodeProtocol cdp = null;
-      
+
       try {
       try {
         cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
         cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
             dfsClient.getConfiguration(), conf.getSocketTimeout(),
             dfsClient.getConfiguration(), conf.getSocketTimeout(),
             conf.isConnectToDnViaHostname(), locatedblock);
             conf.isConnectToDnViaHostname(), locatedblock);
-        
+
         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
         final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
-        
+
         if (n >= 0) {
         if (n >= 0) {
           return n;
           return n;
         }
         }
       }
       }
       catch(IOException ioe) {
       catch(IOException ioe) {
         if (ioe instanceof RemoteException &&
         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
           // special case : replica might not be on the DN, treat as 0 length
           replicaNotFoundCount--;
           replicaNotFoundCount--;
         }
         }
-        
+
         DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
         DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode {}"
               + " for block {}", datanode, locatedblock.getBlock(), ioe);
               + " for block {}", datanode, locatedblock.getBlock(), ioe);
       } finally {
       } finally {
@@ -399,7 +398,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
     throw new IOException("Cannot obtain block length for " + locatedblock);
     throw new IOException("Cannot obtain block length for " + locatedblock);
   }
   }
-  
+
   public long getFileLength() {
   public long getFileLength() {
     synchronized(infoLock) {
     synchronized(infoLock) {
       return locatedBlocks == null? 0:
       return locatedBlocks == null? 0:
@@ -423,7 +422,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
   }
 
 
   /**
   /**
-   * Returns the block containing the target position. 
+   * Returns the block containing the target position.
    */
    */
   synchronized public ExtendedBlock getCurrentBlock() {
   synchronized public ExtendedBlock getCurrentBlock() {
     if (currentLocatedBlock == null){
     if (currentLocatedBlock == null){
@@ -442,7 +441,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /**
   /**
    * Get block at the specified position.
    * Get block at the specified position.
    * Fetch it from the namenode if not cached.
    * Fetch it from the namenode if not cached.
-   * 
+   *
    * @param offset block corresponding to this offset in file is returned
    * @param offset block corresponding to this offset in file is returned
    * @return located block
    * @return located block
    * @throws IOException
    * @throws IOException
@@ -525,12 +524,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         blocks = getFinalizedBlockRange(offset,
         blocks = getFinalizedBlockRange(offset,
           Math.min(length, lengthOfCompleteBlk - offset));
           Math.min(length, lengthOfCompleteBlk - offset));
       } else {
       } else {
-        blocks = new ArrayList<LocatedBlock>(1);
+        blocks = new ArrayList<>(1);
       }
       }
 
 
       // get the blocks from incomplete block range
       // get the blocks from incomplete block range
       if (readLengthPastCompleteBlk) {
       if (readLengthPastCompleteBlk) {
-         blocks.add(locatedBlocks.getLastLocatedBlock());
+        blocks.add(locatedBlocks.getLastLocatedBlock());
       }
       }
 
 
       return blocks;
       return blocks;
@@ -546,7 +545,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       long offset, long length) throws IOException {
       long offset, long length) throws IOException {
     synchronized(infoLock) {
     synchronized(infoLock) {
       assert (locatedBlocks != null) : "locatedBlocks is null";
       assert (locatedBlocks != null) : "locatedBlocks is null";
-      List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
+      List<LocatedBlock> blockRange = new ArrayList<>();
       // search cached blocks first
       // search cached blocks first
       int blockIdx = locatedBlocks.findBlock(offset);
       int blockIdx = locatedBlocks.findBlock(offset);
       if (blockIdx < 0) { // block is not cached
       if (blockIdx < 0) { // block is not cached
@@ -590,10 +589,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     //
     //
     // Connect to best DataNode for desired Block, with potential offset
     // 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 refetchToken = 1; // only need to get a new access token once
     int refetchEncryptionKey = 1; // only need to get a new encryption key once
     int refetchEncryptionKey = 1; // only need to get a new encryption key once
-    
+
     boolean connectFailedOnce = false;
     boolean connectFailedOnce = false;
 
 
     while (true) {
     while (true) {
@@ -638,7 +637,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         } else {
         } else {
           connectFailedOnce = true;
           connectFailedOnce = true;
           DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
           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
           // Put chosen node into dead list, continue
           addToDeadNodes(chosenNode);
           addToDeadNodes(chosenNode);
         }
         }
@@ -721,8 +720,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * strategy-agnostic.
    * strategy-agnostic.
    */
    */
   interface ReaderStrategy {
   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.
      * Copy data from the src ByteBuffer into the read buffer.
@@ -732,7 +731,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
      * @param length Useful only when the ReadStrategy is based on a byte array.
      * @param length Useful only when the ReadStrategy is based on a byte array.
      *               Indicate the length of the data to copy.
      *               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,
   protected void updateReadStatistics(ReadStatistics readStatistics,
@@ -748,7 +747,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       }
       }
     }
     }
   }
   }
-  
+
   /**
   /**
    * Used to read bytes into a byte[]
    * Used to read bytes into a byte[]
    */
    */
@@ -761,7 +760,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
     @Override
     @Override
     public int doRead(BlockReader blockReader, int off, int len)
     public int doRead(BlockReader blockReader, int off, int len)
-          throws ChecksumException, IOException {
+        throws IOException {
       int nRead = blockReader.read(buf, off, len);
       int nRead = blockReader.read(buf, off, len);
       updateReadStatistics(readStatistics, nRead, blockReader);
       updateReadStatistics(readStatistics, nRead, blockReader);
       return nRead;
       return nRead;
@@ -786,7 +785,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
     @Override
     @Override
     public int doRead(BlockReader blockReader, int off, int len)
     public int doRead(BlockReader blockReader, int off, int len)
-        throws ChecksumException, IOException {
+        throws IOException {
       int oldpos = buf.position();
       int oldpos = buf.position();
       int oldlimit = buf.limit();
       int oldlimit = buf.limit();
       boolean success = false;
       boolean success = false;
@@ -804,7 +803,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           buf.position(oldpos);
           buf.position(oldpos);
           buf.limit(oldlimit);
           buf.limit(oldlimit);
         }
         }
-      } 
+      }
     }
     }
 
 
     @Override
     @Override
@@ -820,12 +819,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   /* This is a used by regular read() and handles ChecksumExceptions.
   /* This is a used by regular read() and handles ChecksumExceptions.
    * name readBuffer() is chosen to imply similarity to readBuffer() in
    * name readBuffer() is chosen to imply similarity to readBuffer() in
    * ChecksumFileSystem
    * ChecksumFileSystem
-   */ 
+   */
   private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
   private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
       throws IOException {
     IOException ioe;
     IOException ioe;
-    
+
     /* we retry current node only once. So this is set to true only here.
     /* 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
      * Intention is to handle one common case of an error that is not a
      * failure on datanode or client : when DataNode closes the connection
      * failure on datanode or client : when DataNode closes the connection
@@ -841,7 +840,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       } catch ( ChecksumException ce ) {
       } catch ( ChecksumException ce ) {
         DFSClient.LOG.warn("Found Checksum error for "
         DFSClient.LOG.warn("Found Checksum error for "
             + getCurrentBlock() + " from " + currentNode
             + getCurrentBlock() + " from " + currentNode
-            + " at " + ce.getPos());        
+            + " at " + ce.getPos());
         ioe = ce;
         ioe = ce;
         retryCurrentNode = false;
         retryCurrentNode = false;
         // we want to remember which block replicas we have tried
         // we want to remember which block replicas we have tried
@@ -855,12 +854,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         }
         }
         ioe = e;
         ioe = e;
       }
       }
-      boolean sourceFound = false;
+      boolean sourceFound;
       if (retryCurrentNode) {
       if (retryCurrentNode) {
         /* possibly retry the same node so that transient errors don't
         /* possibly retry the same node so that transient errors don't
          * result in application level failures (e.g. Datanode could have
          * result in application level failures (e.g. Datanode could have
          * closed the connection because the client is idle for too long).
          * closed the connection because the client is idle for too long).
-         */ 
+         */
         sourceFound = seekToBlockSource(pos);
         sourceFound = seekToBlockSource(pos);
       } else {
       } else {
         addToDeadNodes(currentNode);
         addToDeadNodes(currentNode);
@@ -878,8 +877,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if (closed.get()) {
     if (closed.get()) {
       throw new IOException("Stream closed");
       throw new IOException("Stream closed");
     }
     }
-    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap 
-      = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+    Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap = new HashMap<>();
     failures = 0;
     failures = 0;
     if (pos < getFileLength()) {
     if (pos < getFileLength()) {
       int retries = 2;
       int retries = 2;
@@ -898,7 +896,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             }
             }
           }
           }
           int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
           int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
-          
+
           if (result >= 0) {
           if (result >= 0) {
             pos += result;
             pos += result;
           } else {
           } else {
@@ -910,7 +908,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           }
           }
           return result;
           return result;
         } catch (ChecksumException ce) {
         } catch (ChecksumException ce) {
-          throw ce;            
+          throw ce;
         } catch (IOException e) {
         } catch (IOException e) {
           if (retries == 1) {
           if (retries == 1) {
             DFSClient.LOG.warn("DFS Read", e);
             DFSClient.LOG.warn("DFS Read", e);
@@ -923,7 +921,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         } finally {
         } finally {
           // Check if need to report block replicas corruption either read
           // Check if need to report block replicas corruption either read
           // was successful or ChecksumException occured.
           // was successful or ChecksumException occured.
-          reportCheckSumFailure(corruptedBlockMap, 
+          reportCheckSumFailure(corruptedBlockMap,
               currentLocatedBlock.getLocations().length);
               currentLocatedBlock.getLocations().length);
         }
         }
       }
       }
@@ -935,26 +933,21 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Read the entire buffer.
    * Read the entire buffer.
    */
    */
   @Override
   @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);
     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);
       return readWithStrategy(byteArrayReader, off, len);
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
   @Override
   @Override
   public synchronized int read(final ByteBuffer buf) throws IOException {
   public synchronized int read(final ByteBuffer buf) throws IOException {
     ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
     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());
       return readWithStrategy(byteBufferReader, 0, buf.remaining());
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
@@ -964,11 +957,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    */
    */
   protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
   protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
-    Set<DatanodeInfo> dnSet = null;
+    Set<DatanodeInfo> dnSet;
     if((corruptedBlockMap.containsKey(blk))) {
     if((corruptedBlockMap.containsKey(blk))) {
       dnSet = corruptedBlockMap.get(blk);
       dnSet = corruptedBlockMap.get(blk);
     }else {
     }else {
-      dnSet = new HashSet<DatanodeInfo>();
+      dnSet = new HashSet<>();
     }
     }
     if (!dnSet.contains(node)) {
     if (!dnSet.contains(node)) {
       dnSet.add(node);
       dnSet.add(node);
@@ -984,7 +977,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
         return result;
         return result;
       } else {
       } else {
         String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
         String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
-          deadNodes, ignoredNodes);
+            deadNodes, ignoredNodes);
         String blockInfo = block.getBlock() + " file=" + src;
         String blockInfo = block.getBlock() + " file=" + src;
         if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
         if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
           String description = "Could not obtain block: " + blockInfo;
           String description = "Could not obtain block: " + blockInfo;
@@ -1010,7 +1003,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // Also at the second retry, the waiting window is expanded to 6000 ms
           // Also at the second retry, the waiting window is expanded to 6000 ms
           // alleviating the request rate from the server. Similarly the 3rd retry
           // alleviating the request rate from the server. Similarly the 3rd retry
           // will wait 6000ms grace period before retry and the waiting window is
           // will wait 6000ms grace period before retry and the waiting window is
-          // expanded to 9000ms. 
+          // expanded to 9000ms.
           final int timeWindow = dfsClient.getConf().getTimeWindow();
           final int timeWindow = dfsClient.getConf().getTimeWindow();
           double waitTime = timeWindow * failures +       // grace period for the last round of attempt
           double waitTime = timeWindow * failures +       // grace period for the last round of attempt
               // expanding time window for each failure
               // expanding time window for each failure
@@ -1018,7 +1011,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
               ThreadLocalRandom.current().nextDouble();
               ThreadLocalRandom.current().nextDouble();
           DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
           DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
           Thread.sleep((long)waitTime);
           Thread.sleep((long)waitTime);
-        } catch (InterruptedException iex) {
+        } catch (InterruptedException ignored) {
         }
         }
         deadNodes.clear(); //2nd option is to remove only nodes[blockId]
         deadNodes.clear(); //2nd option is to remove only nodes[blockId]
         openInfo(true);
         openInfo(true);
@@ -1130,14 +1123,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       public ByteBuffer call() throws Exception {
       public ByteBuffer call() throws Exception {
         byte[] buf = bb.array();
         byte[] buf = bb.array();
         int offset = bb.position();
         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,
           actualGetFromOneDataNode(datanode, block, start, end, buf,
               offset, corruptedBlockMap);
               offset, corruptedBlockMap);
           return bb;
           return bb;
-        } finally {
-          scope.close();
         }
         }
       }
       }
     };
     };
@@ -1243,12 +1233,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
       throws IOException {
     final DfsClientConf conf = dfsClient.getConf();
     final DfsClientConf conf = dfsClient.getConf();
-    ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
+    ArrayList<Future<ByteBuffer>> futures = new ArrayList<>();
     CompletionService<ByteBuffer> hedgedService =
     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 len = (int) (end - start + 1);
     int hedgedReadId = 0;
     int hedgedReadId = 0;
     block = refreshLocatedBlock(block);
     block = refreshLocatedBlock(block);
@@ -1280,11 +1269,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           // Ignore this node on next go around.
           // Ignore this node on next go around.
           ignored.add(chosenNode.info);
           ignored.add(chosenNode.info);
           dfsClient.getHedgedReadMetrics().incHedgedReadOps();
           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
           // Ignore
-        } catch (ExecutionException e) {
-          // Ignore already logged in the call.
         }
         }
       } else {
       } else {
         // We are starting up a 'hedged' read. We have a read already
         // We are starting up a 'hedged' read. We have a read already
@@ -1349,10 +1336,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       ByteBuffer bb = future.get();
       ByteBuffer bb = future.get();
       futures.remove(future);
       futures.remove(future);
       return bb;
       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
       // already logged in the Callable
       futures.remove(future);
       futures.remove(future);
     }
     }
@@ -1373,7 +1357,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
   /**
   /**
    * Should the block access token be refetched on an exception
    * Should the block access token be refetched on an exception
-   * 
+   *
    * @param ex Exception received
    * @param ex Exception received
    * @param targetAddr Target datanode address from where exception was 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
    * @return true if block access token has expired or invalid and it should be
@@ -1401,23 +1385,20 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
   /**
   /**
    * Read bytes starting from the specified position.
    * Read bytes starting from the specified position.
-   * 
+   *
    * @param position start read from this position
    * @param position start read from this position
    * @param buffer read buffer
    * @param buffer read buffer
    * @param offset offset into buffer
    * @param offset offset into buffer
    * @param length number of bytes to read
    * @param length number of bytes to read
-   * 
+   *
    * @return actual number of bytes read
    * @return actual number of bytes read
    */
    */
   @Override
   @Override
   public int read(long position, byte[] buffer, int offset, int length)
   public int read(long position, byte[] buffer, int offset, int length)
       throws IOException {
       throws IOException {
-    TraceScope scope = dfsClient.
-        newPathTraceScope("DFSInputStream#byteArrayPread", src);
-    try {
+    try (TraceScope ignored = dfsClient.
+        newPathTraceScope("DFSInputStream#byteArrayPread", src)) {
       return pread(position, buffer, offset, length);
       return pread(position, buffer, offset, length);
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
@@ -1437,13 +1418,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     if ((position + length) > filelen) {
     if ((position + length) > filelen) {
       realLen = (int)(filelen - position);
       realLen = (int)(filelen - position);
     }
     }
-    
+
     // determine the block and byte range within the block
     // determine the block and byte range within the block
     // corresponding to position and realLen
     // corresponding to position and realLen
     List<LocatedBlock> blockRange = getBlockRange(position, realLen);
     List<LocatedBlock> blockRange = getBlockRange(position, realLen);
     int remaining = 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) {
     for (LocatedBlock blk : blockRange) {
       long targetStart = position - blk.getStartOffset();
       long targetStart = position - blk.getStartOffset();
       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
@@ -1472,12 +1452,12 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     }
     }
     return realLen;
     return realLen;
   }
   }
-  
+
   /**
   /**
    * DFSInputStream reports checksum failure.
    * DFSInputStream reports checksum failure.
    * Case I : client has tried multiple data nodes and at least one of the
    * 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
    * 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
    * 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
    * 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
    * report otherwise since this maybe due to the client is a handicapped client
@@ -1486,7 +1466,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @param dataNodeCount number of data nodes who contains the block replicas
    * @param dataNodeCount number of data nodes who contains the block replicas
    */
    */
   protected void reportCheckSumFailure(
   protected void reportCheckSumFailure(
-      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap, 
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
       int dataNodeCount) {
       int dataNodeCount) {
     if (corruptedBlockMap.isEmpty()) {
     if (corruptedBlockMap.isEmpty()) {
       return;
       return;
@@ -1553,8 +1533,8 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
           } else {
           } else {
             // The range was already checked. If the block reader returns
             // The range was already checked. If the block reader returns
             // something unexpected instead of throwing an exception, it is
             // 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 + ".";
                 targetPos + ". Instead, it seeked to " + pos + ".";
             DFSClient.LOG.warn(errMsg);
             DFSClient.LOG.warn(errMsg);
             throw new IOException(errMsg);
             throw new IOException(errMsg);
@@ -1580,10 +1560,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     currentNode = blockSeekTo(targetPos);
     currentNode = blockSeekTo(targetPos);
     return true;
     return true;
   }
   }
-  
+
   /**
   /**
    * Seek to given position on a node other than the current node.  If
    * 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.
    * If another node could not be found, then returns false.
    */
    */
   @Override
   @Override
@@ -1596,7 +1576,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     DatanodeInfo oldNode = currentNode;
     DatanodeInfo oldNode = currentNode;
     DatanodeInfo newNode = blockSeekTo(targetPos);
     DatanodeInfo newNode = blockSeekTo(targetPos);
     if (!markedDead) {
     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 and added currentNode again. Thats ok. */
       deadNodes.remove(oldNode);
       deadNodes.remove(oldNode);
     }
     }
@@ -1607,7 +1587,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       return false;
       return false;
     }
     }
   }
   }
-      
+
   /**
   /**
    */
    */
   @Override
   @Override
@@ -1684,7 +1664,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
 
   protected void closeCurrentBlockReaders() {
   protected void closeCurrentBlockReaders() {
     if (blockReader == null) return;
     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.
     // take effect immediately.
     try {
     try {
       blockReader.close();
       blockReader.close();
@@ -1720,11 +1700,11 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * zero-copy read.
    * zero-copy read.
    */
    */
   private static final ByteBuffer EMPTY_BUFFER =
   private static final ByteBuffer EMPTY_BUFFER =
-    ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
+      ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
 
 
   @Override
   @Override
   public synchronized ByteBuffer read(ByteBufferPool bufferPool,
   public synchronized ByteBuffer read(ByteBufferPool bufferPool,
-      int maxLength, EnumSet<ReadOption> opts) 
+      int maxLength, EnumSet<ReadOption> opts)
           throws IOException, UnsupportedOperationException {
           throws IOException, UnsupportedOperationException {
     if (maxLength == 0) {
     if (maxLength == 0) {
       return EMPTY_BUFFER;
       return EMPTY_BUFFER;

+ 71 - 93
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.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -91,7 +90,7 @@ public class DFSOutputStream extends FSOutputSummer
     implements Syncable, CanSetDropBehind {
     implements Syncable, CanSetDropBehind {
   static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
   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).
    * errors (typically related to encryption zones and KeyProvider operations).
    */
    */
   @VisibleForTesting
   @VisibleForTesting
@@ -122,8 +121,9 @@ public class DFSOutputStream extends FSOutputSummer
   private FileEncryptionInfo fileEncryptionInfo;
   private FileEncryptionInfo fileEncryptionInfo;
 
 
   /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
   /** 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 byte[] buf;
     final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
     final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
 
 
@@ -160,9 +160,7 @@ public class DFSOutputStream extends FSOutputSummer
       return null;
       return null;
     }
     }
     DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
     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;
     return value;
   }
   }
 
 
@@ -180,8 +178,8 @@ public class DFSOutputStream extends FSOutputSummer
     return checksum;
     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));
     super(getChecksum4Compute(checksum, stat));
     this.dfsClient = dfsClient;
     this.dfsClient = dfsClient;
     this.src = src;
     this.src = src;
@@ -189,7 +187,7 @@ public class DFSOutputStream extends FSOutputSummer
     this.blockSize = stat.getBlockSize();
     this.blockSize = stat.getBlockSize();
     this.blockReplication = stat.getReplication();
     this.blockReplication = stat.getReplication();
     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
-    this.cachingStrategy = new AtomicReference<CachingStrategy>(
+    this.cachingStrategy = new AtomicReference<>(
         dfsClient.getDefaultWriteCachingStrategy());
         dfsClient.getDefaultWriteCachingStrategy());
     if (progress != null) {
     if (progress != null) {
       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
       DFSClient.LOG.debug("Set non-null progress callback on DFSOutputStream "
@@ -203,21 +201,22 @@ public class DFSOutputStream extends FSOutputSummer
     }
     }
     if (blockSize % bytesPerChecksum != 0) {
     if (blockSize % bytesPerChecksum != 0) {
       throw new HadoopIllegalArgumentException("Invalid values: "
       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();
     this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
   }
   }
 
 
   /** Construct a new output stream for creating a file. */
   /** Construct a new output stream for creating a file. */
-  protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
-      EnumSet<CreateFlag> flag, Progressable progress,
-      DataChecksum checksum, String[] favoredNodes, boolean createStreamer)
-      throws IOException {
+  protected DFSOutputStream(DFSClient dfsClient, String src,
+      HdfsFileStatus stat, EnumSet<CreateFlag> flag, Progressable progress,
+      DataChecksum checksum, String[] favoredNodes, boolean createStreamer) {
     this(dfsClient, src, progress, stat, checksum);
     this(dfsClient, src, progress, stat, checksum);
     this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
     this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
 
 
-    computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
+    computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
+        bytesPerChecksum);
 
 
     if (createStreamer) {
     if (createStreamer) {
       streamer = new DataStreamer(stat, null, dfsClient, src, progress,
       streamer = new DataStreamer(stat, null, dfsClient, src, progress,
@@ -227,11 +226,10 @@ public class DFSOutputStream extends FSOutputSummer
 
 
   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
   static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
       FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
       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 {
       DataChecksum checksum, String[] favoredNodes) throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("newStreamForCreate", src);
-    try {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("newStreamForCreate", src)) {
       HdfsFileStatus stat = null;
       HdfsFileStatus stat = null;
 
 
       // Retry the create if we get a RetryStartFileException up to a maximum
       // Retry the create if we get a RetryStartFileException up to a maximum
@@ -242,7 +240,7 @@ public class DFSOutputStream extends FSOutputSummer
         shouldRetry = false;
         shouldRetry = false;
         try {
         try {
           stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
           stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
-              new EnumSetWritable<CreateFlag>(flag), createParent, replication,
+              new EnumSetWritable<>(flag), createParent, replication,
               blockSize, SUPPORTED_CRYPTO_VERSIONS);
               blockSize, SUPPORTED_CRYPTO_VERSIONS);
           break;
           break;
         } catch (RemoteException re) {
         } catch (RemoteException re) {
@@ -283,8 +281,6 @@ public class DFSOutputStream extends FSOutputSummer
       }
       }
       out.start();
       out.start();
       return out;
       return out;
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
@@ -304,17 +300,17 @@ public class DFSOutputStream extends FSOutputSummer
     // The last partial block of the file has to be filled.
     // The last partial block of the file has to be filled.
     if (!toNewBlock && lastBlock != null) {
     if (!toNewBlock && lastBlock != null) {
       // indicate that we are appending to an existing block
       // 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());
       getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
       adjustPacketChunkSize(stat);
       adjustPacketChunkSize(stat);
       getStreamer().setPipelineInConstruction(lastBlock);
       getStreamer().setPipelineInConstruction(lastBlock);
     } else {
     } else {
       computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
       computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
           bytesPerChecksum);
           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);
     }
     }
   }
   }
 
 
@@ -355,21 +351,19 @@ public class DFSOutputStream extends FSOutputSummer
   }
   }
 
 
   static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
   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);
+      EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
+      HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
+      throws IOException {
     if(stat.getErasureCodingPolicy() != null) {
     if(stat.getErasureCodingPolicy() != null) {
-      throw new IOException("Not support appending to a striping layout file yet.");
+      throw new IOException(
+          "Not support appending to a striping layout file yet.");
     }
     }
-    try {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("newStreamForAppend", src)) {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
           progress, lastBlock, stat, checksum, favoredNodes);
           progress, lastBlock, stat, checksum, favoredNodes);
       out.start();
       out.start();
       return out;
       return out;
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
@@ -486,36 +480,28 @@ public class DFSOutputStream extends FSOutputSummer
    * of the DNs but not necessarily in the DN's OS buffers.
    * of the DNs but not necessarily in the DN's OS buffers.
    *
    *
    * It is a synchronous operation. When it returns,
    * 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.
    * Block allocations are persisted on namenode.
    */
    */
   @Override
   @Override
   public void hflush() throws IOException {
   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));
       flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
   @Override
   @Override
   public void hsync() throws IOException {
   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));
       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).
    * 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.
    * Note that only the current block is flushed to the disk device.
@@ -527,12 +513,8 @@ public class DFSOutputStream extends FSOutputSummer
    *          whether or not to update the block length in NameNode.
    *          whether or not to update the block length in NameNode.
    */
    */
   public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
   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);
       flushOrSync(true, syncFlags);
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
@@ -637,13 +619,14 @@ public class DFSOutputStream extends FSOutputSummer
           dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
           dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
               lastBlockLength);
               lastBlockLength);
         } catch (IOException ioe) {
         } 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();
           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;
           throw ioe;
         }
         }
       }
       }
@@ -654,9 +637,9 @@ public class DFSOutputStream extends FSOutputSummer
         }
         }
       }
       }
     } catch (InterruptedIOException interrupt) {
     } 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;
       throw interrupt;
     } catch (IOException e) {
     } catch (IOException e) {
       DFSClient.LOG.warn("Error while syncing", e);
       DFSClient.LOG.warn("Error while syncing", e);
@@ -698,8 +681,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 {
   protected void flushInternal() throws IOException {
     long toWaitFor;
     long toWaitFor;
@@ -722,7 +705,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.
    * resources associated with this stream.
    */
    */
   synchronized void abort() throws IOException {
   synchronized void abort() throws IOException {
@@ -730,7 +713,7 @@ public class DFSOutputStream extends FSOutputSummer
       return;
       return;
     }
     }
     getStreamer().getLastException().set(new IOException("Lease timeout of "
     getStreamer().getLastException().set(new IOException("Lease timeout of "
-        + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
+        + (dfsClient.getConf().getHdfsTimeout() / 1000) + " seconds expired."));
     closeThreads(true);
     closeThreads(true);
     dfsClient.endFileLease(fileId);
     dfsClient.endFileLease(fileId);
   }
   }
@@ -760,17 +743,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.
    * resources associated with this stream.
    */
    */
   @Override
   @Override
   public synchronized void close() throws IOException {
   public synchronized void close() throws IOException {
-    TraceScope scope =
-        dfsClient.newPathTraceScope("DFSOutputStream#close", src);
-    try {
+    try (TraceScope ignored =
+             dfsClient.newPathTraceScope("DFSOutputStream#close", src)) {
       closeImpl();
       closeImpl();
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
@@ -795,20 +775,18 @@ public class DFSOutputStream extends FSOutputSummer
       // get last block before destroying the streamer
       // get last block before destroying the streamer
       ExtendedBlock lastBlock = getStreamer().getBlock();
       ExtendedBlock lastBlock = getStreamer().getBlock();
       closeThreads(false);
       closeThreads(false);
-      TraceScope scope = dfsClient.getTracer().newScope("completeFile");
-      try {
+      try (TraceScope ignored =
+               dfsClient.getTracer().newScope("completeFile")) {
         completeFile(lastBlock);
         completeFile(lastBlock);
-      } finally {
-        scope.close();
       }
       }
       dfsClient.endFileLease(fileId);
       dfsClient.endFileLease(fileId);
-    } catch (ClosedChannelException e) {
+    } catch (ClosedChannelException ignored) {
     } finally {
     } finally {
       setClosed();
       setClosed();
     }
     }
   }
   }
 
 
-  // should be called holding (this) lock since setTestFilename() may 
+  // should be called holding (this) lock since setTestFilename() may
   // be called during unit tests
   // be called during unit tests
   protected void completeFile(ExtendedBlock last) throws IOException {
   protected void completeFile(ExtendedBlock last) throws IOException {
     long localstart = Time.monotonicNow();
     long localstart = Time.monotonicNow();
@@ -824,12 +802,11 @@ public class DFSOutputStream extends FSOutputSummer
         if (!dfsClient.clientRunning
         if (!dfsClient.clientRunning
             || (hdfsTimeout > 0
             || (hdfsTimeout > 0
                 && localstart + hdfsTimeout < Time.monotonicNow())) {
                 && 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 {
         try {
           if (retries == 0) {
           if (retries == 0) {
@@ -922,9 +899,9 @@ public class DFSOutputStream extends FSOutputSummer
     return getClass().getSimpleName() + ":" + streamer;
     return getClass().getSimpleName() + ":" + streamer;
   }
   }
 
 
-  static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient,
-      String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes)
-      throws IOException {
+  static LocatedBlock addBlock(DatanodeInfo[] excludedNodes,
+      DFSClient dfsClient, String src, ExtendedBlock prevBlock, long fileId,
+      String[] favoredNodes) throws IOException {
     final DfsClientConf conf = dfsClient.getConf();
     final DfsClientConf conf = dfsClient.getConf();
     int retries = conf.getNumBlockWriteLocateFollowingRetry();
     int retries = conf.getNumBlockWriteLocateFollowingRetry();
     long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
     long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
@@ -943,7 +920,8 @@ public class DFSOutputStream extends FSOutputSummer
         if (ue != e) {
         if (ue != e) {
           throw ue; // no need to retry these exceptions
           throw ue; // no need to retry these exceptions
         }
         }
-        if (NotReplicatedYetException.class.getName().equals(e.getClassName())) {
+        if (NotReplicatedYetException.class.getName()
+            .equals(e.getClassName())) {
           if (retries == 0) {
           if (retries == 0) {
             throw e;
             throw e;
           } else {
           } else {

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

@@ -153,7 +153,6 @@ public class DFSPacket {
   /**
   /**
    * Write the full packet, including the header, to the given output stream.
    * Write the full packet, including the header, to the given output stream.
    *
    *
-   * @param stm
    * @throws IOException
    * @throws IOException
    */
    */
   public synchronized void writeTo(DataOutputStream stm) throws IOException {
   public synchronized void writeTo(DataOutputStream stm) throws IOException {
@@ -187,15 +186,18 @@ public class DFSPacket {
 
 
     // corrupt the data for testing.
     // corrupt the data for testing.
     if (DFSClientFaultInjector.get().corruptPacket()) {
     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.
     // 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.
     // undo corruption.
     if (DFSClientFaultInjector.get().uncorruptPacket()) {
     if (DFSClientFaultInjector.get().uncorruptPacket()) {
-      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+      buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^=
+          0xff;
     }
     }
   }
   }
 
 
@@ -207,8 +209,6 @@ public class DFSPacket {
 
 
   /**
   /**
    * Release the buffer in this packet to ByteArrayManager.
    * Release the buffer in this packet to ByteArrayManager.
-   *
-   * @param bam
    */
    */
   synchronized void releaseBuffer(ByteArrayManager bam) {
   synchronized void releaseBuffer(ByteArrayManager bam) {
     bam.release(buf);
     bam.release(buf);

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

@@ -33,7 +33,6 @@ import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult;
 
 
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.io.erasurecode.CodecUtil;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 
 
@@ -586,7 +585,7 @@ public class DFSStripedInputStream extends DFSInputStream {
     abstract void prepareDecodeInputs();
     abstract void prepareDecodeInputs();
 
 
     /** prepare the parity chunk and block reader if necessary */
     /** prepare the parity chunk and block reader if necessary */
-    abstract boolean prepareParityChunk(int index) throws IOException;
+    abstract boolean prepareParityChunk(int index);
 
 
     abstract void decode();
     abstract void decode();
 
 
@@ -878,7 +877,7 @@ public class DFSStripedInputStream extends DFSInputStream {
     }
     }
 
 
     @Override
     @Override
-    boolean prepareParityChunk(int index) throws IOException {
+    boolean prepareParityChunk(int index) {
       Preconditions.checkState(index >= dataBlkNum
       Preconditions.checkState(index >= dataBlkNum
           && alignedStripe.chunks[index] == null);
           && alignedStripe.chunks[index] == null);
       if (blockReaders[index] != null && blockReaders[index].shouldSkip) {
       if (blockReaders[index] != null && blockReaders[index].shouldSkip) {

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

@@ -892,11 +892,9 @@ public class DFSStripedOutputStream extends DFSOutputStream {
       }
       }
 
 
       closeThreads(false);
       closeThreads(false);
-      TraceScope scope = dfsClient.getTracer().newScope("completeFile");
-      try {
+      try (TraceScope ignored =
+               dfsClient.getTracer().newScope("completeFile")) {
         completeFile(currentBlockGroup);
         completeFile(currentBlockGroup);
-      } finally {
-        scope.close();
       }
       }
       dfsClient.endFileLease(fileId);
       dfsClient.endFileLease(fileId);
     } catch (ClosedChannelException ignored) {
     } catch (ClosedChannelException ignored) {

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

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

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

@@ -40,7 +40,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.atomic.AtomicReference;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
@@ -72,7 +71,6 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.core.Sampler;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanId;
 import org.apache.htrace.core.SpanId;
 import org.apache.htrace.core.TraceScope;
 import org.apache.htrace.core.TraceScope;
@@ -87,6 +85,8 @@ import com.google.common.cache.RemovalNotification;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import javax.annotation.Nonnull;
+
 /*********************************************************************
 /*********************************************************************
  *
  *
  * The DataStreamer class is responsible for sending data packets to the
  * The DataStreamer class is responsible for sending data packets to the
@@ -128,7 +128,8 @@ class DataStreamer extends Daemon {
     final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
     final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
     final Socket sock = client.socketFactory.createSocket();
     final Socket sock = client.socketFactory.createSocket();
     final int timeout = client.getDatanodeReadTimeout(length);
     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.setSoTimeout(timeout);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
     sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
     LOG.debug("Send buf size {}", sock.getSendBufferSize());
     LOG.debug("Send buf size {}", sock.getSendBufferSize());
@@ -157,7 +158,7 @@ class DataStreamer extends Daemon {
     }
     }
     packets.clear();
     packets.clear();
   }
   }
-  
+
   class LastExceptionInStreamer {
   class LastExceptionInStreamer {
     private IOException thrown;
     private IOException thrown;
 
 
@@ -448,12 +449,11 @@ class DataStreamer extends Daemon {
    * Construct a data streamer for appending to the last partial block
    * Construct a data streamer for appending to the last partial block
    * @param lastBlock last block of the file to be appended
    * @param lastBlock last block of the file to be appended
    * @param stat status 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,
   DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
                String src, Progressable progress, DataChecksum checksum,
                String src, Progressable progress, DataChecksum checksum,
                AtomicReference<CachingStrategy> cachingStrategy,
                AtomicReference<CachingStrategy> cachingStrategy,
-               ByteArrayManager byteArrayManage) throws IOException {
+               ByteArrayManager byteArrayManage) {
     this(stat, lastBlock.getBlock(), dfsClient, src, progress, checksum, cachingStrategy,
     this(stat, lastBlock.getBlock(), dfsClient, src, progress, checksum, cachingStrategy,
         byteArrayManage, true, null);
         byteArrayManage, true, null);
     stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
     stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
@@ -541,7 +541,7 @@ class DataStreamer extends Daemon {
         // process datanode IO errors if any
         // process datanode IO errors if any
         boolean doSleep = processDatanodeOrExternalError();
         boolean doSleep = processDatanodeOrExternalError();
 
 
-        final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; 
+        final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2;
         synchronized (dataQueue) {
         synchronized (dataQueue) {
           // wait for a packet to be sent.
           // wait for a packet to be sent.
           long now = Time.monotonicNow();
           long now = Time.monotonicNow();
@@ -644,9 +644,8 @@ class DataStreamer extends Daemon {
         LOG.debug(this + " sending " + one);
         LOG.debug(this + " sending " + one);
 
 
         // write out data to remote datanode
         // 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);
           one.writeTo(blockStream);
           blockStream.flush();
           blockStream.flush();
         } catch (IOException e) {
         } catch (IOException e) {
@@ -658,8 +657,6 @@ class DataStreamer extends Daemon {
           // will be taken out then.
           // will be taken out then.
           errorState.markFirstNodeIfNotMarked();
           errorState.markFirstNodeIfNotMarked();
           throw e;
           throw e;
-        } finally {
-          writeScope.close();
         }
         }
         lastPacket = Time.monotonicNow();
         lastPacket = Time.monotonicNow();
 
 
@@ -749,9 +746,8 @@ class DataStreamer extends Daemon {
    * @throws IOException
    * @throws IOException
    */
    */
   void waitForAckedSeqno(long seqno) 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);
       LOG.debug("Waiting for ack for: {}", seqno);
       long begin = Time.monotonicNow();
       long begin = Time.monotonicNow();
       try {
       try {
@@ -771,15 +767,13 @@ class DataStreamer extends Daemon {
           }
           }
         }
         }
         checkClosed();
         checkClosed();
-      } catch (ClosedChannelException e) {
+      } catch (ClosedChannelException cce) {
       }
       }
       long duration = Time.monotonicNow() - begin;
       long duration = Time.monotonicNow() - begin;
       if (duration > dfsclientSlowLogThresholdMs) {
       if (duration > dfsclientSlowLogThresholdMs) {
         LOG.warn("Slow waitForAckedSeqno took " + duration
         LOG.warn("Slow waitForAckedSeqno took " + duration
             + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
             + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
       }
       }
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
@@ -826,7 +820,7 @@ class DataStreamer extends Daemon {
         }
         }
         checkClosed();
         checkClosed();
         queuePacket(packet);
         queuePacket(packet);
-      } catch (ClosedChannelException e) {
+      } catch (ClosedChannelException ignored) {
       }
       }
     }
     }
   }
   }
@@ -928,10 +922,7 @@ class DataStreamer extends Daemon {
       assert false;
       assert false;
     }
     }
 
 
-    if (addr != null && NetUtils.isLocalAddress(addr)) {
-      return true;
-    }
-    return false;
+    return addr != null && NetUtils.isLocalAddress(addr);
   }
   }
 
 
   //
   //
@@ -1169,17 +1160,15 @@ class DataStreamer extends Daemon {
   ) throws IOException {
   ) throws IOException {
     if (nodes.length != original.length + 1) {
     if (nodes.length != original.length + 1) {
       throw new IOException(
       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++) {
     for(int i = 0; i < nodes.length; i++) {
       int j = 0;
       int j = 0;
@@ -1228,7 +1217,7 @@ class DataStreamer extends Daemon {
     final StorageType[] originalTypes = storageTypes;
     final StorageType[] originalTypes = storageTypes;
     final String[] originalIDs = storageIDs;
     final String[] originalIDs = storageIDs;
     IOException caughtException = null;
     IOException caughtException = null;
-    ArrayList<DatanodeInfo> exclude = new ArrayList<DatanodeInfo>(failed);
+    ArrayList<DatanodeInfo> exclude = new ArrayList<>(failed);
     while (tried < 3) {
     while (tried < 3) {
       LocatedBlock lb;
       LocatedBlock lb;
       //get a new datanode
       //get a new datanode
@@ -1267,7 +1256,8 @@ class DataStreamer extends Daemon {
 
 
   private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
   private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
                         final StorageType[] targetStorageTypes,
                         final StorageType[] targetStorageTypes,
-                        final Token<BlockTokenIdentifier> blockToken) throws IOException {
+                        final Token<BlockTokenIdentifier> blockToken)
+      throws IOException {
     //transfer replica to the new datanode
     //transfer replica to the new datanode
     Socket sock = null;
     Socket sock = null;
     DataOutputStream out = null;
     DataOutputStream out = null;
@@ -1278,7 +1268,8 @@ class DataStreamer extends Daemon {
 
 
       // transfer timeout multiplier based on the transfer size
       // transfer timeout multiplier based on the transfer size
       // One per 200 packets = 12.8MB. Minimum is 2.
       // 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);
       final long readTimeout = dfsClient.getDatanodeReadTimeout(multi);
 
 
       OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
       OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
@@ -1448,12 +1439,14 @@ class DataStreamer extends Daemon {
         // good reports should follow bad ones, if client committed
         // good reports should follow bad ones, if client committed
         // with those nodes.
         // with those nodes.
         Thread.sleep(2000);
         Thread.sleep(2000);
-      } catch (InterruptedException ie) {}
+      } catch (InterruptedException ignored) {
+      }
     }
     }
   }
   }
 
 
   private LocatedBlock updateBlockForPipeline() throws IOException {
   private LocatedBlock updateBlockForPipeline() throws IOException {
-    return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
+    return dfsClient.namenode.updateBlockForPipeline(block,
+        dfsClient.clientName);
   }
   }
 
 
   static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
   static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) {
@@ -1480,11 +1473,11 @@ class DataStreamer extends Daemon {
    * Returns the list of target datanodes.
    * Returns the list of target datanodes.
    */
    */
   protected LocatedBlock nextBlockOutputStream() throws IOException {
   protected LocatedBlock nextBlockOutputStream() throws IOException {
-    LocatedBlock lb = null;
-    DatanodeInfo[] nodes = null;
-    StorageType[] storageTypes = null;
-    int count = getNumBlockWriteRetry();
-    boolean success = false;
+    LocatedBlock lb;
+    DatanodeInfo[] nodes;
+    StorageType[] storageTypes;
+    int count = dfsClient.getConf().getNumBlockWriteRetry();
+    boolean success;
     ExtendedBlock oldBlock = block;
     ExtendedBlock oldBlock = block;
     do {
     do {
       errorState.resetInternalError();
       errorState.resetInternalError();
@@ -1534,7 +1527,6 @@ class DataStreamer extends Daemon {
       LOG.info("nodes are empty for write pipeline of " + block);
       LOG.info("nodes are empty for write pipeline of " + block);
       return false;
       return false;
     }
     }
-    Status pipelineStatus = SUCCESS;
     String firstBadLink = "";
     String firstBadLink = "";
     boolean checkRestart = false;
     boolean checkRestart = false;
     if (LOG.isDebugEnabled()) {
     if (LOG.isDebugEnabled()) {
@@ -1569,25 +1561,26 @@ class DataStreamer extends Daemon {
         // Xmit header info to datanode
         // 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
         // We cannot change the block length in 'block' as it counts the number
         // of bytes ack'ed.
         // of bytes ack'ed.
         ExtendedBlock blockCopy = new ExtendedBlock(block);
         ExtendedBlock blockCopy = new ExtendedBlock(block);
         blockCopy.setNumBytes(stat.getBlockSize());
         blockCopy.setNumBytes(stat.getBlockSize());
 
 
-        boolean[] targetPinnings = getPinnings(nodes, true);
+        boolean[] targetPinnings = getPinnings(nodes);
         // send the request
         // send the request
         new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
         new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
             dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
             dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
             nodes.length, block.getNumBytes(), bytesSent, newGS,
             nodes.length, block.getNumBytes(), bytesSent, newGS,
             checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
             checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
-            (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
+            (targetPinnings != null && targetPinnings[0]), targetPinnings);
 
 
         // receive ack for connect
         // receive ack for connect
         BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
         BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
             PBHelperClient.vintPrefixed(blockReplyStream));
             PBHelperClient.vintPrefixed(blockReplyStream));
-        pipelineStatus = resp.getStatus();
+        Status pipelineStatus = resp.getStatus();
         firstBadLink = resp.getFirstBadLink();
         firstBadLink = resp.getFirstBadLink();
 
 
         // Got an restart OOB ack.
         // Got an restart OOB ack.
@@ -1600,7 +1593,7 @@ class DataStreamer extends Daemon {
           checkRestart = true;
           checkRestart = true;
           throw new IOException("A datanode is restarting.");
           throw new IOException("A datanode is restarting.");
         }
         }
-		
+
         String logInfo = "ack with firstBadLink as " + firstBadLink;
         String logInfo = "ack with firstBadLink as " + firstBadLink;
         DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
         DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
 
 
@@ -1612,7 +1605,8 @@ class DataStreamer extends Daemon {
         if (!errorState.isRestartingNode()) {
         if (!errorState.isRestartingNode()) {
           LOG.info("Exception in createBlockOutputStream " + this, ie);
           LOG.info("Exception in createBlockOutputStream " + this, ie);
         }
         }
-        if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+        if (ie instanceof InvalidEncryptionKeyException &&
+            refetchEncryptionKey > 0) {
           LOG.info("Will fetch a new encryption key and retry, "
           LOG.info("Will fetch a new encryption key and retry, "
               + "encryption key was invalid when connecting to "
               + "encryption key was invalid when connecting to "
               + nodes[0] + " : " + ie);
               + nodes[0] + " : " + ie);
@@ -1634,14 +1628,15 @@ class DataStreamer extends Daemon {
             }
             }
           }
           }
         } else {
         } else {
-          assert checkRestart == false;
+          assert !checkRestart;
           errorState.setBadNodeIndex(0);
           errorState.setBadNodeIndex(0);
         }
         }
 
 
         final int i = errorState.getBadNodeIndex();
         final int i = errorState.getBadNodeIndex();
         // Check whether there is a restart worth waiting for.
         // Check whether there is a restart worth waiting for.
         if (checkRestart && shouldWaitForRestart(i)) {
         if (checkRestart && shouldWaitForRestart(i)) {
-          errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
+          errorState.initRestartingNode(i, "Datanode " + i + " is restarting: "
+              + nodes[i]);
         }
         }
         errorState.setInternalError();
         errorState.setInternalError();
         lastException.set(ie);
         lastException.set(ie);
@@ -1651,7 +1646,6 @@ class DataStreamer extends Daemon {
           IOUtils.closeSocket(s);
           IOUtils.closeSocket(s);
           s = null;
           s = null;
           IOUtils.closeStream(out);
           IOUtils.closeStream(out);
-          out = null;
           IOUtils.closeStream(blockReplyStream);
           IOUtils.closeStream(blockReplyStream);
           blockReplyStream = null;
           blockReplyStream = null;
         }
         }
@@ -1660,19 +1654,18 @@ class DataStreamer extends Daemon {
     }
     }
   }
   }
 
 
-  private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
+  private boolean[] getPinnings(DatanodeInfo[] nodes) {
     if (favoredNodes == null) {
     if (favoredNodes == null) {
       return null;
       return null;
     } else {
     } else {
       boolean[] pinnings = new boolean[nodes.length];
       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++) {
       for (int i = 0; i < nodes.length; i++) {
         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
         pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
         LOG.debug("{} was chosen by name node (favored={}).",
         LOG.debug("{} was chosen by name node (favored={}).",
             nodes[i].getXferAddrWithHostname(), pinnings[i]);
             nodes[i].getXferAddrWithHostname(), pinnings[i]);
       }
       }
-      if (shouldLog && !favoredSet.isEmpty()) {
+      if (!favoredSet.isEmpty()) {
         // There is one or more favored nodes that were not allocated.
         // There is one or more favored nodes that were not allocated.
         LOG.warn("These favored nodes were specified but not chosen: "
         LOG.warn("These favored nodes were specified but not chosen: "
             + favoredSet + " Specified favored nodes: "
             + favoredSet + " Specified favored nodes: "
@@ -1782,7 +1775,7 @@ class DataStreamer extends Daemon {
    * For heartbeat packets, create buffer directly by new byte[]
    * For heartbeat packets, create buffer directly by new byte[]
    * since heartbeats should not be blocked.
    * since heartbeats should not be blocked.
    */
    */
-  private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
+  private DFSPacket createHeartbeatPacket() {
     final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
     final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
     return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
     return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
   }
   }
@@ -1794,7 +1787,8 @@ class DataStreamer extends Daemon {
         .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
         .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
           @Override
           @Override
           public void onRemoval(
           public void onRemoval(
-              RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
+              @Nonnull RemovalNotification<DatanodeInfo, DatanodeInfo>
+                  notification) {
             LOG.info("Removing node " + notification.getKey()
             LOG.info("Removing node " + notification.getKey()
                 + " from the excluded nodes list");
                 + " from the excluded nodes list");
           }
           }

File diff suppressed because it is too large
+ 142 - 185
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) {
   public static ExtendedBlockId fromExtendedBlock(ExtendedBlock block) {
     return new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
     return new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
   }
   }
-  
+
   public ExtendedBlockId(long blockId, String bpId) {
   public ExtendedBlockId(long blockId, String bpId) {
     this.blockId = blockId;
     this.blockId = blockId;
     this.bpId = bpId;
     this.bpId = bpId;
@@ -76,7 +76,6 @@ final public class ExtendedBlockId {
 
 
   @Override
   @Override
   public String toString() {
   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

@@ -80,7 +80,7 @@ public final class ExternalBlockReader implements BlockReader {
   }
   }
 
 
   @Override
   @Override
-  public int available() throws IOException {
+  public int available() {
     // We return the amount of bytes between the current offset and the visible
     // 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
     // length.  Some of the other block readers return a shorter length than
     // that.  The only advantage to returning a shorter length is that the
     // 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;
 package org.apache.hadoop.hdfs;
 
 
-import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
@@ -37,10 +36,13 @@ import com.google.common.cache.RemovalNotification;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import javax.annotation.Nonnull;
+
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class KeyProviderCache {
 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;
   private final Cache<URI, KeyProvider> cache;
 
 
@@ -50,14 +52,14 @@ public class KeyProviderCache {
         .removalListener(new RemovalListener<URI, KeyProvider>() {
         .removalListener(new RemovalListener<URI, KeyProvider>() {
           @Override
           @Override
           public void onRemoval(
           public void onRemoval(
-              RemovalNotification<URI, KeyProvider> notification) {
+              @Nonnull RemovalNotification<URI, KeyProvider> notification) {
             try {
             try {
+              assert notification.getValue() != null;
               notification.getValue().close();
               notification.getValue().close();
             } catch (Throwable e) {
             } catch (Throwable e) {
               LOG.error(
               LOG.error(
                   "Error closing KeyProvider with uri ["
                   "Error closing KeyProvider with uri ["
                       + notification.getKey() + "]", e);
                       + notification.getKey() + "]", e);
-              ;
             }
             }
           }
           }
         })
         })
@@ -83,8 +85,8 @@ public class KeyProviderCache {
   }
   }
 
 
   private URI createKeyProviderURI(Configuration conf) {
   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
     // No provider set in conf
     if (providerUriStr.isEmpty()) {
     if (providerUriStr.isEmpty()) {
       LOG.error("Could not find uri with key ["
       LOG.error("Could not find uri with key ["
@@ -104,9 +106,9 @@ public class KeyProviderCache {
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
-  public void setKeyProvider(Configuration conf, KeyProvider keyProvider)
-      throws IOException {
+  public void setKeyProvider(Configuration conf, KeyProvider keyProvider) {
     URI uri = createKeyProviderURI(conf);
     URI uri = createKeyProviderURI(conf);
+    assert uri != null;
     cache.put(uri, keyProvider);
     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
    * @param failoverProxyProvider Failover proxy provider
    * @return an object containing both the proxy and the associated
    * @return an object containing both the proxy and the associated
    *         delegation token service it corresponds to
    *         delegation token service it corresponds to
-   * @throws IOException
    */
    */
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   public static <T> ProxyAndInfo<T> createHAProxy(
   public static <T> ProxyAndInfo<T> createHAProxy(
       Configuration conf, URI nameNodeUri, Class<T> xface,
       Configuration conf, URI nameNodeUri, Class<T> xface,
-      AbstractNNFailoverProxyProvider<T> failoverProxyProvider)
-      throws IOException {
+      AbstractNNFailoverProxyProvider<T> failoverProxyProvider) {
     Preconditions.checkNotNull(failoverProxyProvider);
     Preconditions.checkNotNull(failoverProxyProvider);
     // HA case
     // HA case
     DfsClientConf config = new DfsClientConf(conf);
     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.net.Peer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.util.IOUtilsClient;
 import org.apache.hadoop.hdfs.util.IOUtilsClient;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Time;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
@@ -46,16 +45,16 @@ import org.slf4j.LoggerFactory;
 @VisibleForTesting
 @VisibleForTesting
 public class PeerCache {
 public class PeerCache {
   private static final Logger LOG = LoggerFactory.getLogger(PeerCache.class);
   private static final Logger LOG = LoggerFactory.getLogger(PeerCache.class);
-  
+
   private static class Key {
   private static class Key {
     final DatanodeID dnID;
     final DatanodeID dnID;
     final boolean isDomain;
     final boolean isDomain;
-    
+
     Key(DatanodeID dnID, boolean isDomain) {
     Key(DatanodeID dnID, boolean isDomain) {
       this.dnID = dnID;
       this.dnID = dnID;
       this.isDomain = isDomain;
       this.isDomain = isDomain;
     }
     }
-    
+
     @Override
     @Override
     public boolean equals(Object o) {
     public boolean equals(Object o) {
       if (!(o instanceof Key)) {
       if (!(o instanceof Key)) {
@@ -70,7 +69,7 @@ public class PeerCache {
       return dnID.hashCode() ^ (isDomain ? 1 : 0);
       return dnID.hashCode() ^ (isDomain ? 1 : 0);
     }
     }
   }
   }
-  
+
   private static class Value {
   private static class Value {
     private final Peer peer;
     private final Peer peer;
     private final long time;
     private final long time;
@@ -92,10 +91,10 @@ public class PeerCache {
   private Daemon daemon;
   private Daemon daemon;
   /** A map for per user per datanode. */
   /** A map for per user per datanode. */
   private final LinkedListMultimap<Key, Value> multimap =
   private final LinkedListMultimap<Key, Value> multimap =
-    LinkedListMultimap.create();
+      LinkedListMultimap.create();
   private final int capacity;
   private final int capacity;
   private final long expiryPeriod;
   private final long expiryPeriod;
-  
+
   public PeerCache(int c, long e) {
   public PeerCache(int c, long e) {
     this.capacity = c;
     this.capacity = c;
     this.expiryPeriod = e;
     this.expiryPeriod = e;
@@ -107,17 +106,17 @@ public class PeerCache {
          expiryPeriod + " when cache is enabled.");
          expiryPeriod + " when cache is enabled.");
     }
     }
   }
   }
- 
+
   private boolean isDaemonStarted() {
   private boolean isDaemonStarted() {
-    return (daemon == null)? false: true;
+    return daemon != null;
   }
   }
 
 
   private synchronized void startExpiryDaemon() {
   private synchronized void startExpiryDaemon() {
     // start daemon only if not already started
     // start daemon only if not already started
-    if (isDaemonStarted() == true) {
+    if (isDaemonStarted()) {
       return;
       return;
     }
     }
-    
+
     daemon = new Daemon(new Runnable() {
     daemon = new Daemon(new Runnable() {
       @Override
       @Override
       public void run() {
       public void run() {
@@ -144,7 +143,7 @@ public class PeerCache {
    * @param isDomain     Whether to retrieve a DomainPeer or not.
    * @param isDomain     Whether to retrieve a DomainPeer or not.
    *
    *
    * @return             An open Peer connected to the DN, or null if none
    * @return             An open Peer connected to the DN, or null if none
-   *                     was found. 
+   *                     was found.
    */
    */
   public Peer get(DatanodeID dnId, boolean isDomain) {
   public Peer get(DatanodeID dnId, boolean isDomain) {
 
 
@@ -215,12 +214,11 @@ public class PeerCache {
   private synchronized void evictExpired(long expiryPeriod) {
   private synchronized void evictExpired(long expiryPeriod) {
     while (multimap.size() != 0) {
     while (multimap.size() != 0) {
       Iterator<Entry<Key, Value>> iter =
       Iterator<Entry<Key, Value>> iter =
-        multimap.entries().iterator();
+          multimap.entries().iterator();
       Entry<Key, Value> entry = iter.next();
       Entry<Key, Value> entry = iter.next();
       // if oldest socket expired, remove it
       // 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;
         break;
       }
       }
       IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
       IOUtilsClient.cleanup(LOG, entry.getValue().getPeer());
@@ -235,8 +233,7 @@ public class PeerCache {
     // We can get the oldest element immediately, because of an interesting
     // We can get the oldest element immediately, because of an interesting
     // property of LinkedListMultimap: its iterator traverses entries in the
     // property of LinkedListMultimap: its iterator traverses entries in the
     // order that they were added.
     // order that they were added.
-    Iterator<Entry<Key, Value>> iter =
-      multimap.entries().iterator();
+    Iterator<Entry<Key, Value>> iter = multimap.entries().iterator();
     if (!iter.hasNext()) {
     if (!iter.hasNext()) {
       throw new IllegalStateException("Cannot evict from empty cache! " +
       throw new IllegalStateException("Cannot evict from empty cache! " +
         "capacity: " + capacity);
         "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 {
   private void run() throws InterruptedException {
     for(long lastExpiryTime = Time.monotonicNow();
     for(long lastExpiryTime = Time.monotonicNow();
@@ -274,7 +271,7 @@ public class PeerCache {
     }
     }
     multimap.clear();
     multimap.clear();
   }
   }
-  
+
   @VisibleForTesting
   @VisibleForTesting
   void close() {
   void close() {
     clear();
     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
  * @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.
  * It will be removed in the next release.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
@@ -79,7 +80,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   private final long blockId;
   private final long blockId;
 
 
   /** offset in block of of first chunk - may be less than startOffset
   /** 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 long firstChunkOffset;
 
 
   private final int bytesPerChecksum;
   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.
    * at the beginning so that the read can begin on a chunk boundary.
    */
    */
   private final long bytesNeededToFinish;
   private final long bytesNeededToFinish;
-  
+
   /**
   /**
    * True if we are reading from a local DataNode.
    * 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 eos = false;
   private boolean sentStatusCode = false;
   private boolean sentStatusCode = false;
-  
+
   ByteBuffer checksumBytes = null;
   ByteBuffer checksumBytes = null;
   /** Amount of unread data in the current received packet */
   /** Amount of unread data in the current received packet */
   int dataLeft = 0;
   int dataLeft = 0;
-  
+
   private final PeerCache peerCache;
   private final PeerCache peerCache;
 
 
   private final Tracer tracer;
   private final Tracer tracer;
-  
+
   /* FSInputChecker interface */
   /* FSInputChecker interface */
-  
+
   /* same interface as inputStream java.io.InputStream#read()
   /* same interface as inputStream java.io.InputStream#read()
    * used by DFSInputStream#read()
    * used by DFSInputStream#read()
    * This violates one rule when there is a checksum error:
    * This violates one rule when there is a checksum error:
@@ -118,9 +119,9 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
    * the checksum.
    * the checksum.
    */
    */
   @Override
   @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
     // This has to be set here, *before* the skip, since we can
     // hit EOS during the skip, in the case that our entire read
     // hit EOS during the skip, in the case that our entire read
     // is smaller than the checksum chunk.
     // 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");
         throw new IOException("Could not skip required number of bytes");
       }
       }
     }
     }
-    
+
     int nRead = super.read(buf, off, len);
     int nRead = super.read(buf, off, len);
 
 
     // if eos was set in the previous read, send a status code to the DN
     // 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
   @Override
   public synchronized long skip(long n) throws IOException {
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
     /* 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;
     long nSkipped = 0;
     while (nSkipped < n) {
     while (nSkipped < n) {
       int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
       int toSkip = (int)Math.min(n-nSkipped, Integer.MAX_VALUE);
@@ -168,18 +169,18 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   @Override
   @Override
   public int read() throws IOException {
   public int read() throws IOException {
     throw new IOException("read() is not expected to be invoked. " +
     throw new IOException("read() is not expected to be invoked. " +
-                          "Use read(buf, off, len) instead.");
+        "Use read(buf, off, len) instead.");
   }
   }
-  
+
   @Override
   @Override
   public boolean seekToNewSource(long targetPos) throws IOException {
   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.
      * case of pread(), it just tries a different replica without seeking.
-     */ 
+     */
     return false;
     return false;
   }
   }
-  
+
   @Override
   @Override
   public void seek(long pos) throws IOException {
   public void seek(long pos) throws IOException {
     throw new IOException("Seek() is not supported in BlockInputChecker");
     throw new IOException("Seek() is not supported in BlockInputChecker");
@@ -188,17 +189,17 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   @Override
   @Override
   protected long getChunkPosition(long pos) {
   protected long getChunkPosition(long pos) {
     throw new RuntimeException("getChunkPosition() is not supported, " +
     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.
    * to be read.
    */
    */
   private void adjustChecksumBytes(int dataLen) {
   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()) {
     if (checksumBytes == null || requiredSize > checksumBytes.capacity()) {
       checksumBytes =  ByteBuffer.wrap(new byte[requiredSize]);
       checksumBytes =  ByteBuffer.wrap(new byte[requiredSize]);
     } else {
     } else {
@@ -206,42 +207,39 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     }
     }
     checksumBytes.limit(requiredSize);
     checksumBytes.limit(requiredSize);
   }
   }
-  
+
   @Override
   @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);
       return readChunkImpl(pos, buf, offset, len, checksumBuf);
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 
   private synchronized int readChunkImpl(long pos, byte[] buf, int offset,
   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.
     // Read one chunk.
     if (eos) {
     if (eos) {
       // Already hit EOF
       // Already hit EOF
       return -1;
       return -1;
     }
     }
-    
+
     // Read one DATA_CHUNK.
     // Read one DATA_CHUNK.
     long chunkOffset = lastChunkOffset;
     long chunkOffset = lastChunkOffset;
     if ( lastChunkLen > 0 ) {
     if ( lastChunkLen > 0 ) {
       chunkOffset += lastChunkLen;
       chunkOffset += lastChunkLen;
     }
     }
-    
+
     // pos is relative to the start of the first chunk of the read.
     // pos is relative to the start of the first chunk of the read.
     // chunkOffset is relative to the start of the block.
     // chunkOffset is relative to the start of the block.
     // This makes sure that the read passed from FSInputChecker is the
     // This makes sure that the read passed from FSInputChecker is the
     // for the same chunk we expect to be reading from the DN.
     // for the same chunk we expect to be reading from the DN.
     if ( (pos + firstChunkOffset) != chunkOffset ) {
     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.
     // 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
       // Sanity check the lengths
       if (!header.sanityCheck(lastSeqNo)) {
       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();
       lastSeqNo = header.getSeqno();
@@ -263,7 +261,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       adjustChecksumBytes(header.getDataLen());
       adjustChecksumBytes(header.getDataLen());
       if (header.getDataLen() > 0) {
       if (header.getDataLen() > 0) {
         IOUtils.readFully(in, checksumBytes.array(), 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
       // How many chunks we can fit in databuffer
       //  - note this is a floor since we always read full chunks
       //  - note this is a floor since we always read full chunks
       int chunksCanFit = Math.min(len / bytesPerChecksum,
       int chunksCanFit = Math.min(len / bytesPerChecksum,
-                                  checksumBuf.length / checksumSize);
+          checksumBuf.length / checksumSize);
 
 
       // How many chunks should we read
       // How many chunks should we read
       checksumsToRead = Math.min(chunksLeft, chunksCanFit);
       checksumsToRead = Math.min(chunksLeft, chunksCanFit);
       // How many bytes should we actually read
       // How many bytes should we actually read
       bytesToRead = Math.min(
       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 {
     } else {
       // no checksum
       // no checksum
       bytesToRead = Math.min(dataLeft, len);
       bytesToRead = Math.min(dataLeft, len);
@@ -328,7 +326,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
       if (!hdr.isLastPacketInBlock() ||
       if (!hdr.isLastPacketInBlock() ||
           hdr.getDataLen() != 0) {
           hdr.getDataLen() != 0) {
         throw new IOException("Expected empty end-of-read packet! Header: " +
         throw new IOException("Expected empty end-of-read packet! Header: " +
-                              hdr);
+            hdr);
       }
       }
 
 
       eos = true;
       eos = true;
@@ -340,22 +338,22 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
 
 
     return bytesToRead;
     return bytesToRead;
   }
   }
-  
+
   private RemoteBlockReader(String file, String bpid, long blockId,
   private RemoteBlockReader(String file, String bpid, long blockId,
       DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
       DataInputStream in, DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
     // Path is used only for printing block and file information in debug
     // Path is used only for printing block and file information in debug
     super(new Path("/" + Block.BLOCK_FILE_PREFIX + blockId +
     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.
     this.isLocal = DFSUtilClient.isLocalAddress(NetUtils.
         createSocketAddr(datanodeID.getXferAddr()));
         createSocketAddr(datanodeID.getXferAddr()));
-    
+
     this.peer = peer;
     this.peer = peer;
     this.datanodeID = datanodeID;
     this.datanodeID = datanodeID;
     this.in = in;
     this.in = in;
@@ -394,46 +392,46 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
    * @return New BlockReader instance, or null on error.
    * @return New BlockReader instance, or null on error.
    */
    */
   public static RemoteBlockReader newBlockReader(String file,
   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)
     // in and out will be closed when sock is closed (by the caller)
     final DataOutputStream out =
     final DataOutputStream out =
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
         new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
         verifyChecksum, cachingStrategy);
         verifyChecksum, cachingStrategy);
-    
+
     //
     //
     // Get bytes in block, set streams
     // Get bytes in block, set streams
     //
     //
 
 
     DataInputStream in = new DataInputStream(
     DataInputStream in = new DataInputStream(
         new BufferedInputStream(peer.getInputStream(), bufferSize));
         new BufferedInputStream(peer.getInputStream(), bufferSize));
-    
+
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
     BlockOpResponseProto status = BlockOpResponseProto.parseFrom(
         PBHelperClient.vintPrefixed(in));
         PBHelperClient.vintPrefixed(in));
     RemoteBlockReader2.checkSuccess(status, peer, block, file);
     RemoteBlockReader2.checkSuccess(status, peer, block, file);
     ReadOpChecksumInfoProto checksumInfo =
     ReadOpChecksumInfoProto checksumInfo =
-      status.getReadOpChecksumInfo();
+        status.getReadOpChecksumInfo();
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
         checksumInfo.getChecksum());
         checksumInfo.getChecksum());
     //Warning when we get CHECKSUM_NULL?
     //Warning when we get CHECKSUM_NULL?
-    
+
     // Read the first chunk offset.
     // Read the first chunk offset.
     long firstChunkOffset = checksumInfo.getChunkOffset();
     long firstChunkOffset = checksumInfo.getChunkOffset();
-    
+
     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
       throw new IOException("BlockReader: error in first chunk offset (" +
       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(),
     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.
     // in will be closed when its Socket is closed.
   }
   }
-  
+
   @Override
   @Override
   public void readFully(byte[] buf, int readOffset, int amtToRead)
   public void readFully(byte[] buf, int readOffset, int amtToRead)
       throws IOException {
       throws IOException {
@@ -479,7 +477,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
     } catch (IOException e) {
     } catch (IOException e) {
       // It's ok not to be able to send this. But something is probably wrong.
       // 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 " +
       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 {
   public int read(ByteBuffer buf) throws IOException {
     throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
     throw new UnsupportedOperationException("readDirect unsupported in RemoteBlockReader");
   }
   }
-  
+
   @Override
   @Override
-  public int available() throws IOException {
+  public int available() {
     // An optimistic estimate of how much data is available
     // An optimistic estimate of how much data is available
     // to us without doing network I/O.
     // to us without doing network I/O.
     return RemoteBlockReader2.TCP_WINDOW_SIZE;
     return RemoteBlockReader2.TCP_WINDOW_SIZE;
@@ -499,7 +497,7 @@ public class RemoteBlockReader extends FSInputChecker implements BlockReader {
   public boolean isLocal() {
   public boolean isLocal() {
     return isLocal;
     return isLocal;
   }
   }
-  
+
   @Override
   @Override
   public boolean isShortCircuit() {
   public boolean isShortCircuit() {
     return false;
     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() {
   public Peer getPeer() {
     return peer;
     return peer;
   }
   }
-  
+
   @Override
   @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);
     UUID randomId = (LOG.isTraceEnabled() ? UUID.randomUUID() : null);
     LOG.trace("Starting read #{} file {} from datanode {}",
     LOG.trace("Starting read #{} file {} from datanode {}",
         randomId, filename, datanodeID.getHostName());
         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();
         readNextPacket();
-      } finally {
-        scope.close();
       }
       }
     }
     }
 
 
@@ -155,23 +153,21 @@ public class RemoteBlockReader2  implements BlockReader {
       // we're at EOF now
       // we're at EOF now
       return -1;
       return -1;
     }
     }
-    
+
     int nRead = Math.min(curDataSlice.remaining(), len);
     int nRead = Math.min(curDataSlice.remaining(), len);
     curDataSlice.get(buf, off, nRead);
     curDataSlice.get(buf, off, nRead);
-    
+
     return nRead;
     return nRead;
   }
   }
 
 
 
 
   @Override
   @Override
   public synchronized int read(ByteBuffer buf) throws IOException {
   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();
         readNextPacket();
-      } finally {
-        scope.close();
       }
       }
     }
     }
     if (curDataSlice.remaining() == 0) {
     if (curDataSlice.remaining() == 0) {
@@ -195,23 +191,24 @@ public class RemoteBlockReader2  implements BlockReader {
     PacketHeader curHeader = packetReceiver.getHeader();
     PacketHeader curHeader = packetReceiver.getHeader();
     curDataSlice = packetReceiver.getDataSlice();
     curDataSlice = packetReceiver.getDataSlice();
     assert curDataSlice.capacity() == curHeader.getDataLen();
     assert curDataSlice.capacity() == curHeader.getDataLen();
-    
+
     LOG.trace("DFSClient readNextPacket got header {}", curHeader);
     LOG.trace("DFSClient readNextPacket got header {}", curHeader);
 
 
     // Sanity check the lengths
     // Sanity check the lengths
     if (!curHeader.sanityCheck(lastSeqNo)) {
     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) {
     if (curHeader.getDataLen() > 0) {
       int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum;
       int chunks = 1 + (curHeader.getDataLen() - 1) / bytesPerChecksum;
       int checksumsLen = chunks * checksumSize;
       int checksumsLen = chunks * checksumSize;
 
 
       assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
       assert packetReceiver.getChecksumSlice().capacity() == checksumsLen :
-        "checksum slice capacity=" + packetReceiver.getChecksumSlice().capacity() + 
-          " checksumsLen=" + checksumsLen;
-      
+          "checksum slice capacity=" +
+              packetReceiver.getChecksumSlice().capacity() +
+              " checksumsLen=" + checksumsLen;
+
       lastSeqNo = curHeader.getSeqno();
       lastSeqNo = curHeader.getSeqno();
       if (verifyChecksum && curDataSlice.remaining() > 0) {
       if (verifyChecksum && curDataSlice.remaining() > 0) {
         // N.B.: the checksum error offset reported here is actually
         // N.B.: the checksum error offset reported here is actually
@@ -223,8 +220,8 @@ public class RemoteBlockReader2  implements BlockReader {
             filename, curHeader.getOffsetInBlock());
             filename, curHeader.getOffsetInBlock());
       }
       }
       bytesNeededToFinish -= curHeader.getDataLen();
       bytesNeededToFinish -= curHeader.getDataLen();
-    }    
-    
+    }
+
     // First packet will include some data prior to the first byte
     // First packet will include some data prior to the first byte
     // the user requested. Skip it.
     // the user requested. Skip it.
     if (curHeader.getOffsetInBlock() < startOffset) {
     if (curHeader.getOffsetInBlock() < startOffset) {
@@ -243,7 +240,7 @@ public class RemoteBlockReader2  implements BlockReader {
       }
       }
     }
     }
   }
   }
-  
+
   @Override
   @Override
   public synchronized long skip(long n) throws IOException {
   public synchronized long skip(long n) throws IOException {
     /* How can we make sure we don't throw a ChecksumException, at least
     /* 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;
     long skipped = 0;
     while (skipped < n) {
     while (skipped < n) {
       long needToSkip = n - skipped;
       long needToSkip = n - skipped;
-      if (curDataSlice == null || curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
+      if (curDataSlice == null ||
+          curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
         readNextPacket();
         readNextPacket();
       }
       }
       if (curDataSlice.remaining() == 0) {
       if (curDataSlice.remaining() == 0) {
@@ -273,13 +271,13 @@ public class RemoteBlockReader2  implements BlockReader {
 
 
     PacketHeader trailer = packetReceiver.getHeader();
     PacketHeader trailer = packetReceiver.getHeader();
     if (!trailer.isLastPacketInBlock() ||
     if (!trailer.isLastPacketInBlock() ||
-       trailer.getDataLen() != 0) {
+        trailer.getDataLen() != 0) {
       throw new IOException("Expected empty end-of-read packet! Header: " +
       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,
       DataChecksum checksum, boolean verifyChecksum,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       long startOffset, long firstChunkOffset, long bytesToRead, Peer peer,
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
       DatanodeID datanodeID, PeerCache peerCache, Tracer tracer) {
@@ -320,7 +318,7 @@ public class RemoteBlockReader2  implements BlockReader {
 
 
     // in will be closed when its Socket is closed.
     // in will be closed when its Socket is closed.
   }
   }
-  
+
   /**
   /**
    * When the reader reaches end of the read, it sends a status response
    * 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
    * (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) {
     } catch (IOException e) {
       // It's ok not to be able to send this. But something is probably wrong.
       // 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 " +
       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)
   static void writeReadResult(OutputStream out, Status statusCode)
       throws IOException {
       throws IOException {
-    
+
     ClientReadStatusProto.newBuilder()
     ClientReadStatusProto.newBuilder()
-      .setStatus(statusCode)
-      .build()
-      .writeDelimitedTo(out);
+        .setStatus(statusCode)
+        .build()
+        .writeDelimitedTo(out);
 
 
     out.flush();
     out.flush();
   }
   }
-  
+
   /**
   /**
    * File name to print when accessing a block directly (from servlets)
    * File name to print when accessing a block directly (from servlets)
    * @param s Address of the block location
    * @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 {
   public void readFully(byte[] buf, int off, int len) throws IOException {
     BlockReaderUtil.readFully(this, buf, off, len);
     BlockReaderUtil.readFully(this, buf, off, len);
   }
   }
-  
+
   /**
   /**
    * Create a new BlockReader specifically to satisfy a read.
    * Create a new BlockReader specifically to satisfy a read.
    * This method also sends the OP_READ_BLOCK request.
    * 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.
    * @return New BlockReader instance, or null on error.
    */
    */
   public static BlockReader newBlockReader(String file,
   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)
     // in and out will be closed when sock is closed (by the caller)
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
-          peer.getOutputStream()));
+        peer.getOutputStream()));
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
     new Sender(out).readBlock(block, blockToken, clientName, startOffset, len,
         verifyChecksum, cachingStrategy);
         verifyChecksum, cachingStrategy);
 
 
@@ -415,7 +413,7 @@ public class RemoteBlockReader2  implements BlockReader {
         PBHelperClient.vintPrefixed(in));
         PBHelperClient.vintPrefixed(in));
     checkSuccess(status, peer, block, file);
     checkSuccess(status, peer, block, file);
     ReadOpChecksumInfoProto checksumInfo =
     ReadOpChecksumInfoProto checksumInfo =
-      status.getReadOpChecksumInfo();
+        status.getReadOpChecksumInfo();
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
     DataChecksum checksum = DataTransferProtoUtil.fromProto(
         checksumInfo.getChecksum());
         checksumInfo.getChecksum());
     //Warning when we get CHECKSUM_NULL?
     //Warning when we get CHECKSUM_NULL?
@@ -426,13 +424,13 @@ public class RemoteBlockReader2  implements BlockReader {
     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
     if ( firstChunkOffset < 0 || firstChunkOffset > startOffset ||
         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
         firstChunkOffset <= (startOffset - checksum.getBytesPerChecksum())) {
       throw new IOException("BlockReader: error in first chunk offset (" +
       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(
   static void checkSuccess(
@@ -440,26 +438,26 @@ public class RemoteBlockReader2  implements BlockReader {
       ExtendedBlock block, String file)
       ExtendedBlock block, String file)
       throws IOException {
       throws IOException {
     String logInfo = "for OP_READ_BLOCK"
     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);
     DataTransferProtoUtil.checkBlockOpStatus(status, logInfo);
   }
   }
-  
+
   @Override
   @Override
-  public int available() throws IOException {
+  public int available() {
     // An optimistic estimate of how much data is available
     // An optimistic estimate of how much data is available
     // to us without doing network I/O.
     // to us without doing network I/O.
     return TCP_WINDOW_SIZE;
     return TCP_WINDOW_SIZE;
   }
   }
-  
+
   @Override
   @Override
   public boolean isLocal() {
   public boolean isLocal() {
     return isLocal;
     return isLocal;
   }
   }
-  
+
   @Override
   @Override
   public boolean isShortCircuit() {
   public boolean isShortCircuit() {
     return false;
     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
    * @param datanodeId    ID of destination DataNode
    * @return              A new Peer connected to the address.
    * @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.
    *                      the remote socket, encrypted stream, etc.
    */
    */
   Peer newConnectedPeer(InetSocketAddress addr,
   Peer newConnectedPeer(InetSocketAddress addr,

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

@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 
 

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

+ 30 - 19
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;
   long    DFS_BLOCK_SIZE_DEFAULT = 128*1024*1024;
   String  DFS_REPLICATION_KEY = "dfs.replication";
   String  DFS_REPLICATION_KEY = "dfs.replication";
   short   DFS_REPLICATION_DEFAULT = 3;
   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_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
   String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
   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})?)*$";
       "^(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})?)*$";
@@ -121,7 +122,8 @@ public interface HdfsClientConfigKeys {
       "dfs.datanode.hdfs-blocks-metadata.enabled";
       "dfs.datanode.hdfs-blocks-metadata.enabled";
   boolean DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT = false;
   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";
   String  DFS_DATANODE_READAHEAD_BYTES_KEY = "dfs.datanode.readahead.bytes";
   long    DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
   long    DFS_DATANODE_READAHEAD_BYTES_DEFAULT = 4 * 1024 * 1024; // 4MB
   String  DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
   String  DFS_ENCRYPTION_KEY_PROVIDER_URI = "dfs.encryption.key.provider.uri";
@@ -138,7 +140,8 @@ public interface HdfsClientConfigKeys {
       "dfs.encrypt.data.transfer.cipher.key.bitlength";
       "dfs.encrypt.data.transfer.cipher.key.bitlength";
   int    DFS_ENCRYPT_DATA_TRANSFER_CIPHER_KEY_BITLENGTH_DEFAULT = 128;
   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 =
   String REPLICA_ACCESSOR_BUILDER_CLASSES_KEY =
       PREFIX + "replica.accessor.builder.classes";
       PREFIX + "replica.accessor.builder.classes";
@@ -157,13 +160,15 @@ public interface HdfsClientConfigKeys {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
     String PREFIX = HdfsClientConfigKeys.PREFIX + "retry.";
 
 
     String  POLICY_ENABLED_KEY = PREFIX + "policy.enabled";
     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_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;
     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;
     int     INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT = 4000;
 
 
     String  MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
     String  MAX_ATTEMPTS_KEY = PREFIX + "max.attempts";
@@ -186,17 +191,19 @@ public interface HdfsClientConfigKeys {
     int     SLEEPTIME_MAX_DEFAULT = 15000;
     int     SLEEPTIME_MAX_DEFAULT = 15000;
     String  CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
     String  CONNECTION_RETRIES_KEY = PREFIX + "connection.retries";
     int     CONNECTION_RETRIES_DEFAULT = 0;
     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;
     int     CONNECTION_RETRIES_ON_SOCKET_TIMEOUTS_DEFAULT = 0;
   }
   }
-  
+
   /** dfs.client.write configuration properties */
   /** dfs.client.write configuration properties */
   interface Write {
   interface Write {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "write.";
     String PREFIX = HdfsClientConfigKeys.PREFIX + "write.";
 
 
     String  MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
     String  MAX_PACKETS_IN_FLIGHT_KEY = PREFIX + "max-packets-in-flight";
     int     MAX_PACKETS_IN_FLIGHT_DEFAULT = 80;
     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;
     long    EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT = 10*MINUTE;
 
 
     interface ByteArrayManager {
     interface ByteArrayManager {
@@ -208,7 +215,8 @@ public interface HdfsClientConfigKeys {
       int     COUNT_THRESHOLD_DEFAULT = 128;
       int     COUNT_THRESHOLD_DEFAULT = 128;
       String  COUNT_LIMIT_KEY = PREFIX + "count-limit";
       String  COUNT_LIMIT_KEY = PREFIX + "count-limit";
       int     COUNT_LIMIT_DEFAULT = 2048;
       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;
       long    COUNT_RESET_TIME_PERIOD_MS_DEFAULT = 10*SECOND;
     }
     }
   }
   }
@@ -219,9 +227,11 @@ public interface HdfsClientConfigKeys {
 
 
     String  RETRIES_KEY = PREFIX + "retries";
     String  RETRIES_KEY = PREFIX + "retries";
     int     RETRIES_DEFAULT = 3;
     int     RETRIES_DEFAULT = 3;
-    String  LOCATEFOLLOWINGBLOCK_RETRIES_KEY = PREFIX + "locateFollowingBlock.retries";
+    String  LOCATEFOLLOWINGBLOCK_RETRIES_KEY =
+        PREFIX + "locateFollowingBlock.retries";
     int     LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT = 5;
     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;
     int     LOCATEFOLLOWINGBLOCK_INITIAL_DELAY_MS_DEFAULT = 400;
 
 
     interface ReplaceDatanodeOnFailure {
     interface ReplaceDatanodeOnFailure {
@@ -239,8 +249,8 @@ public interface HdfsClientConfigKeys {
   /** dfs.client.read configuration properties */
   /** dfs.client.read configuration properties */
   interface Read {
   interface Read {
     String PREFIX = HdfsClientConfigKeys.PREFIX + "read.";
     String PREFIX = HdfsClientConfigKeys.PREFIX + "read.";
-    
-    String  PREFETCH_SIZE_KEY = PREFIX + "prefetch.size"; 
+
+    String  PREFETCH_SIZE_KEY = PREFIX + "prefetch.size";
 
 
     interface ShortCircuit {
     interface ShortCircuit {
       String PREFIX = Read.PREFIX + "shortcircuit.";
       String PREFIX = Read.PREFIX + "shortcircuit.";
@@ -263,7 +273,8 @@ public interface HdfsClientConfigKeys {
   interface ShortCircuit {
   interface ShortCircuit {
     String PREFIX = Read.PREFIX + "short.circuit.";
     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;
     long    REPLICA_STALE_THRESHOLD_MS_DEFAULT = 30*MINUTE;
   }
   }
 
 
@@ -295,8 +306,8 @@ public interface HdfsClientConfigKeys {
 
 
     String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
     String  THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size";
     /**
     /**
-     * With default RS-6-3-64k erasure coding policy, each normal read could span
-     * 6 DNs, so this default value accommodates 3 read streams
+     * With default RS-6-3-64k erasure coding policy, each normal read could
+     * span 6 DNs, so this default value accommodates 3 read streams
      */
      */
     int     THREADPOOL_SIZE_DEFAULT = 18;
     int     THREADPOOL_SIZE_DEFAULT = 18;
   }
   }
@@ -312,7 +323,7 @@ public interface HdfsClientConfigKeys {
     String  RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
     String  RETRY_POLICY_SPEC_DEFAULT = "10000,6,60000,10"; //t1,n1,t2,n2,...
     String  RETRY_MAX_ATTEMPTS_KEY = PREFIX + "retry.max.attempts";
     String  RETRY_MAX_ATTEMPTS_KEY = PREFIX + "retry.max.attempts";
     int     RETRY_MAX_ATTEMPTS_DEFAULT = 10;
     int     RETRY_MAX_ATTEMPTS_DEFAULT = 10;
-    
+
     // failover
     // failover
     String  FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "failover.max.attempts";
     String  FAILOVER_MAX_ATTEMPTS_KEY = PREFIX + "failover.max.attempts";
     int     FAILOVER_MAX_ATTEMPTS_DEFAULT =  15;
     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
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class HdfsDataInputStream extends FSDataInputStream {
 public class HdfsDataInputStream extends FSDataInputStream {
-  public HdfsDataInputStream(DFSInputStream in) throws IOException {
+  public HdfsDataInputStream(DFSInputStream in) {
     super(in);
     super(in);
   }
   }
 
 
-  public HdfsDataInputStream(CryptoInputStream in) throws IOException {
+  public HdfsDataInputStream(CryptoInputStream in) {
     super(in);
     super(in);
     Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
     Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
         "CryptoInputStream should wrap a DFSInputStream");
         "CryptoInputStream should wrap a DFSInputStream");
@@ -63,7 +63,7 @@ public class HdfsDataInputStream extends FSDataInputStream {
    * @return the underlying output stream
    * @return the underlying output stream
    */
    */
   public InputStream getWrappedStream() {
   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
    * Get the visible length of the file. It will include the length of the last
    * block even if that is in UnderConstruction state.
    * block even if that is in UnderConstruction state.
-   * 
+   *
    * @return The visible length of the file.
    * @return The visible length of the file.
    */
    */
-  public long getVisibleLength() throws IOException {
+  public long getVisibleLength() {
     return getDFSInputStream().getFileLength();
     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);
     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);
     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);
     super(out, stats, startPosition);
-    Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
+    Preconditions.checkArgument(
+        out.getWrappedStream() instanceof DFSOutputStream,
         "CryptoOutputStream should wrap a 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);
     this(out, stats, 0L);
   }
   }
 
 
   /**
   /**
    * Get the actual number of replicas of the current block.
    * Get the actual number of replicas of the current block.
-   * 
+   *
    * This can be different from the designated replication factor of the file
    * This can be different from the designated replication factor of the file
    * because the namenode does not maintain replication for the blocks which are
    * because the namenode does not maintain replication for the blocks which are
    * currently being written to. Depending on the configuration, the client may
    * 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
    * 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
    * have failed, or the client may add a new datanodes once a datanode has
    * failed.
    * failed.
-   * 
+   *
    * @return the number of valid replicas of the current block
    * @return the number of valid replicas of the current block
    */
    */
   public synchronized int getCurrentBlockReplication() throws IOException {
   public synchronized int getCurrentBlockReplication() throws IOException {
@@ -77,10 +78,10 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
     }
     }
     return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
     return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
   }
   }
-  
+
   /**
   /**
    * Sync buffered data to DataNodes (flush to disk devices).
    * Sync buffered data to DataNodes (flush to disk devices).
-   * 
+   *
    * @param syncFlags
    * @param syncFlags
    *          Indicate the detailed semantic and actions of the hsync.
    *          Indicate the detailed semantic and actions of the hsync.
    * @throws IOException
    * @throws IOException
@@ -89,13 +90,13 @@ public class HdfsDataOutputStream extends FSDataOutputStream {
   public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
   public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
     OutputStream wrappedStream = getWrappedStream();
     OutputStream wrappedStream = getWrappedStream();
     if (wrappedStream instanceof CryptoOutputStream) {
     if (wrappedStream instanceof CryptoOutputStream) {
-      ((CryptoOutputStream) wrappedStream).flush();
+      wrappedStream.flush();
       wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
       wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
     }
     }
     ((DFSOutputStream) wrappedStream).hsync(syncFlags);
     ((DFSOutputStream) wrappedStream).hsync(syncFlags);
   }
   }
-  
-  public static enum SyncFlag {
+
+  public enum SyncFlag {
 
 
     /**
     /**
      * When doing sync to DataNodes, also update the metadata (block length) in
      * 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
      * Sync the data to DataNode, close the current block, and allocate a new
      * block
      * 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
   @Override
   public boolean hasNext() {
   public boolean hasNext() {
     return nextPath != null;
     return nextPath != null;
   }
   }
 
 
-  
+
   @Override
   @Override
   public Path next() throws IOException {
   public Path next() throws IOException {
     if (!hasNext()) {
     if (!hasNext()) {
@@ -102,4 +102,4 @@ public class CorruptFileBlockIterator implements RemoteIterator<Path> {
 
 
     return result;
     return result;
   }
   }
-}
+}

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

@@ -80,7 +80,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.ShortCircuit;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.Write;
 
 
-import java.lang.Class;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.List;
 import java.util.List;
@@ -252,9 +251,8 @@ public class DfsClientConf {
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   private List<Class<? extends ReplicaAccessorBuilder>>
   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);
         HdfsClientConfigKeys.REPLICA_ACCESSOR_BUILDER_CLASSES_KEY);
     if (classNames.length == 0) {
     if (classNames.length == 0) {
       return Collections.emptyList();
       return Collections.emptyList();
@@ -265,8 +263,8 @@ public class DfsClientConf {
     for (String className: classNames) {
     for (String className: classNames) {
       try {
       try {
         Class<? extends ReplicaAccessorBuilder> cls =
         Class<? extends ReplicaAccessorBuilder> cls =
-          (Class<? extends ReplicaAccessorBuilder>)
-            classLoader.loadClass(className);
+            (Class<? extends ReplicaAccessorBuilder>)
+                classLoader.loadClass(className);
         classes.add(cls);
         classes.add(cls);
       } catch (Throwable t) {
       } catch (Throwable t) {
         LOG.warn("Unable to load " + className, 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>
  * <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),
  * When a file is opened for write (create or append),
  * namenode stores a file lease for recording the identity of the writer.
  * 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.
  * The writer (i.e. the DFSClient) is required to renew the lease periodically.
@@ -57,7 +57,8 @@ import org.slf4j.LoggerFactory;
  * <li>
  * <li>
  * It maintains a map from (namenode, user) pairs to lease renewers.
  * It maintains a map from (namenode, user) pairs to lease renewers.
  * The same {@link LeaseRenewer} instance is used for renewing lease
  * 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>
  * <li>
  * <li>
  * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
  * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
@@ -80,7 +81,7 @@ public class LeaseRenewer {
 
 
   /** Get a {@link LeaseRenewer} instance */
   /** Get a {@link LeaseRenewer} instance */
   public static LeaseRenewer getInstance(final String authority,
   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);
     final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
     r.addClient(dfsc);
     r.addClient(dfsc);
     return r;
     return r;
@@ -136,7 +137,7 @@ public class LeaseRenewer {
     }
     }
 
 
     /** A map for per user per namenode renewers. */
     /** 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. */
     /** Get a renewer. */
     private synchronized LeaseRenewer get(final String authority,
     private synchronized LeaseRenewer get(final String authority,
@@ -189,7 +190,7 @@ public class LeaseRenewer {
   private final Factory.Key factorykey;
   private final Factory.Key factorykey;
 
 
   /** A list of clients corresponding to this renewer. */
   /** 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
    * A stringified stack trace of the call stack when the Lease Renewer
@@ -404,7 +405,7 @@ public class LeaseRenewer {
   private void renew() throws IOException {
   private void renew() throws IOException {
     final List<DFSClient> copies;
     final List<DFSClient> copies;
     synchronized(this) {
     synchronized(this) {
-      copies = new ArrayList<DFSClient>(dfsclients);
+      copies = new ArrayList<>(dfsclients);
     }
     }
     //sort the client names for finding out repeated names.
     //sort the client names for finding out repeated names.
     Collections.sort(copies, new Comparator<DFSClient>() {
     Collections.sort(copies, new Comparator<DFSClient>() {
@@ -414,8 +415,7 @@ public class LeaseRenewer {
       }
       }
     });
     });
     String previousName = "";
     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.
       //skip if current client name is the same as the previous name.
       if (!c.getClientName().equals(previousName)) {
       if (!c.getClientName().equals(previousName)) {
         if (!c.renewLease()) {
         if (!c.renewLease()) {
@@ -470,7 +470,7 @@ public class LeaseRenewer {
               LOG.debug("Lease renewer daemon for " + clientsString()
               LOG.debug("Lease renewer daemon for " + clientsString()
                   + " with renew id " + id + " is not current");
                   + " with renew id " + id + " is not current");
             } else {
             } else {
-               LOG.debug("Lease renewer daemon for " + clientsString()
+              LOG.debug("Lease renewer daemon for " + clientsString()
                   + " with renew id " + id + " expired");
                   + " 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
  * See the License for the specific language governing permissions and
  * limitations under the License.
  * 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
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public abstract class Event {
 public abstract class Event {
-  public static enum EventType {
+  public enum EventType {
     CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK, TRUNCATE
     CREATE, CLOSE, APPEND, RENAME, METADATA, UNLINK, TRUNCATE
   }
   }
 
 
@@ -98,8 +98,8 @@ public abstract class Event {
   @InterfaceAudience.Public
   @InterfaceAudience.Public
   public static class CreateEvent extends Event {
   public static class CreateEvent extends Event {
 
 
-    public static enum INodeType {
-      FILE, DIRECTORY, SYMLINK;
+    public enum INodeType {
+      FILE, DIRECTORY, SYMLINK
     }
     }
 
 
     private INodeType iNodeType;
     private INodeType iNodeType;
@@ -247,17 +247,21 @@ public abstract class Event {
     @InterfaceStability.Unstable
     @InterfaceStability.Unstable
     public String toString() {
     public String toString() {
       StringBuilder content = new StringBuilder();
       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) {
       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();
       return content.toString();
     }
     }
 
 
@@ -274,8 +278,8 @@ public abstract class Event {
   @InterfaceAudience.Public
   @InterfaceAudience.Public
   public static class MetadataUpdateEvent extends Event {
   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;
     private String path;
@@ -434,28 +438,29 @@ public abstract class Event {
     @InterfaceStability.Unstable
     @InterfaceStability.Unstable
     public String toString() {
     public String toString() {
       StringBuilder content = new StringBuilder();
       StringBuilder content = new StringBuilder();
-      content.append("MetadataUpdateEvent [path=" + path + ", metadataType="
-          + metadataType);
+      content.append("MetadataUpdateEvent [path=").append(path)
+          .append(", metadataType=").append(metadataType);
       switch (metadataType) {
       switch (metadataType) {
       case TIMES:
       case TIMES:
-        content.append(", mtime=" + mtime + ", atime=" + atime);
+        content.append(", mtime=").append(mtime)
+            .append(", atime=").append(atime);
         break;
         break;
       case REPLICATION:
       case REPLICATION:
-        content.append(", replication=" + replication);
+        content.append(", replication=").append(replication);
         break;
         break;
       case OWNER:
       case OWNER:
-        content.append(", ownerName=" + ownerName
-            + ", groupName=" + groupName);
+        content.append(", ownerName=").append(ownerName)
+            .append(", groupName=").append(groupName);
         break;
         break;
       case PERMS:
       case PERMS:
-        content.append(", perms=" + perms);
+        content.append(", perms=").append(perms);
         break;
         break;
       case ACLS:
       case ACLS:
-        content.append(", acls=" + acls);
+        content.append(", acls=").append(acls);
         break;
         break;
       case XATTRS:
       case XATTRS:
-        content.append(", xAttrs=" + xAttrs + ", xAttrsRemoved="
-            + xAttrsRemoved);
+        content.append(", xAttrs=").append(xAttrs)
+            .append(", xAttrsRemoved=").append(xAttrsRemoved);
         break;
         break;
       default:
       default:
         break;
         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
   @Override
   public void setWriteTimeout(int timeoutMs) {
   public void setWriteTimeout(int timeoutMs) {
-   /* 
+   /*
     * We can't implement write timeouts. :(
     * We can't implement write timeouts. :(
-    * 
+    *
     * Java provides no facility to set a blocking write timeout on a Socket.
     * Java provides no facility to set a blocking write timeout on a Socket.
     * You can simulate a blocking write with a timeout by using
     * 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
     * non-blocking I/O.  However, we can't use nio here, because this Socket
     * doesn't have an associated Channel.
     * doesn't have an associated Channel.
-    * 
+    *
     * See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
     * See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4031100 for
     * more details.
     * more details.
     */
     */
@@ -100,7 +100,7 @@ public class BasicInetPeer implements Peer {
   public String getLocalAddressString() {
   public String getLocalAddressString() {
     return socket.getLocalSocketAddress().toString();
     return socket.getLocalSocketAddress().toString();
   }
   }
-  
+
   @Override
   @Override
   public InputStream getInputStream() throws IOException {
   public InputStream getInputStream() throws IOException {
     return in;
     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;
 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.
  * on a UNIX domain socket.
  */
  */
 @InterfaceAudience.Private
 @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.
    * An encrypted InputStream.
    */
    */
   private final InputStream in;
   private final InputStream in;
-  
+
   /**
   /**
    * An encrypted OutputStream.
    * An encrypted OutputStream.
    */
    */
   private final OutputStream out;
   private final OutputStream out;
-  
+
   /**
   /**
    * An encrypted ReadableByteChannel.
    * An encrypted ReadableByteChannel.
    */
    */
@@ -53,7 +53,7 @@ public class EncryptedPeer implements Peer {
     this.enclosedPeer = enclosedPeer;
     this.enclosedPeer = enclosedPeer;
     this.in = ios.in;
     this.in = ios.in;
     this.out = ios.out;
     this.out = ios.out;
-    this.channel = ios.in instanceof ReadableByteChannel ? 
+    this.channel = ios.in instanceof ReadableByteChannel ?
         (ReadableByteChannel)ios.in : null;
         (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;
 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.
  * on a Socket.
  */
  */
 public class NioInetPeer implements Peer {
 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.
    * An InputStream which simulates blocking I/O with timeouts using NIO.
    */
    */
   private final SocketInputStream in;
   private final SocketInputStream in;
-  
+
   /**
   /**
    * An OutputStream which simulates blocking I/O with timeouts using NIO.
    * 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
    * @return                The input stream channel associated with this
    *                        peer, or null if it has none.
    *                        peer, or null if it has none.
    */
    */
-  public ReadableByteChannel getInputStreamChannel();
+  ReadableByteChannel getInputStreamChannel();
 
 
   /**
   /**
    * Set the read timeout on this peer.
    * Set the read timeout on this peer.
    *
    *
    * @param timeoutMs       The timeout in milliseconds.
    * @param timeoutMs       The timeout in milliseconds.
    */
    */
-  public void setReadTimeout(int timeoutMs) throws IOException;
+  void setReadTimeout(int timeoutMs) throws IOException;
 
 
   /**
   /**
    * @return                The receive buffer size.
    * @return                The receive buffer size.
    */
    */
-  public int getReceiveBufferSize() throws IOException;
+  int getReceiveBufferSize() throws IOException;
 
 
   /**
   /**
    * @return                True if TCP_NODELAY is turned on.
    * @return                True if TCP_NODELAY is turned on.
    */
    */
-  public boolean getTcpNoDelay() throws IOException;
+  boolean getTcpNoDelay() throws IOException;
 
 
   /**
   /**
    * Set the write timeout on this peer.
    * Set the write timeout on this peer.
@@ -61,63 +61,63 @@ public interface Peer extends Closeable {
    *
    *
    * @param timeoutMs       The timeout in milliseconds.
    * @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.
    * @return                true only if the peer is closed.
    */
    */
-  public boolean isClosed();
-  
+  boolean isClosed();
+
   /**
   /**
    * Close the peer.
    * Close the peer.
    *
    *
    * It's safe to re-close a Peer that is already closed.
    * 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
    * @return               A string representing the remote end of our
    *                       connection to the peer.
    *                       connection to the peer.
    */
    */
-  public String getRemoteAddressString();
+  String getRemoteAddressString();
 
 
   /**
   /**
    * @return               A string representing the local end of our
    * @return               A string representing the local end of our
    *                       connection to the peer.
    *                       connection to the peer.
    */
    */
-  public String getLocalAddressString();
-  
+  String getLocalAddressString();
+
   /**
   /**
    * @return               An InputStream associated with the Peer.
    * @return               An InputStream associated with the Peer.
    *                       This InputStream will be valid until you close
    *                       This InputStream will be valid until you close
    *                       this peer with Peer#close.
    *                       this peer with Peer#close.
    */
    */
-  public InputStream getInputStream() throws IOException;
-  
+  InputStream getInputStream() throws IOException;
+
   /**
   /**
    * @return               An OutputStream associated with the Peer.
    * @return               An OutputStream associated with the Peer.
    *                       This OutputStream will be valid until you close
    *                       This OutputStream will be valid until you close
    *                       this peer with Peer#close.
    *                       this peer with Peer#close.
    */
    */
-  public OutputStream getOutputStream() throws IOException;
+  OutputStream getOutputStream() throws IOException;
 
 
   /**
   /**
    * @return               True if the peer resides on the same
    * @return               True if the peer resides on the same
    *                       computer as we.
    *                       computer as we.
    */
    */
-  public boolean isLocal();
+  boolean isLocal();
 
 
   /**
   /**
    * @return               The DomainSocket associated with the current
    * @return               The DomainSocket associated with the current
    *                       peer, or null if there is none.
    *                       peer, or null if there is none.
    */
    */
-  public DomainSocket getDomainSocket();
-  
+  DomainSocket getDomainSocket();
+
   /**
   /**
    * Return true if the channel is secure.
    * Return true if the channel is secure.
    *
    *
    * @return               True if our channel to this peer is not
    * @return               True if our channel to this peer is not
    *                       susceptible to man-in-the-middle attacks.
    *                       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.classification.InterfaceStability;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.io.*;
 
 
+import javax.annotation.Nonnull;
+
 /**************************************************
 /**************************************************
  * A Block is a Hadoop FS primitive, identified by a
  * A Block is a Hadoop FS primitive, identified by a
  * long.
  * long.
@@ -36,12 +38,10 @@ public class Block implements Writable, Comparable<Block> {
   public static final String BLOCK_FILE_PREFIX = "blk_";
   public static final String BLOCK_FILE_PREFIX = "blk_";
   public static final String METADATA_EXTENSION = ".meta";
   public static final String METADATA_EXTENSION = ".meta";
   static {                                      // register a ctor
   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
   public static final Pattern blockFilePattern = Pattern
@@ -208,20 +208,14 @@ public class Block implements Writable, Comparable<Block> {
   }
   }
 
 
   @Override // Comparable
   @Override // Comparable
-  public int compareTo(Block b) {
+  public int compareTo(@Nonnull Block b) {
     return blockId < b.blockId ? -1 :
     return blockId < b.blockId ? -1 :
-           blockId > b.blockId ? 1 : 0;
+        blockId > b.blockId ? 1 : 0;
   }
   }
 
 
   @Override // Object
   @Override // Object
   public boolean equals(Object o) {
   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) {
   public static boolean matchingIdAndGenStamp(Block a, Block b) {
     if (a == b) return true; // same block, or both null
     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
   @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.
    * 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 file Block data file.
    * @param metafile Metadata file for the block.
    * @param metafile Metadata file for the block.
    */
    */
@@ -48,12 +48,12 @@ public class BlockLocalPathInfo {
    * @return Block data file.
    * @return Block data file.
    */
    */
   public String getBlockPath() {return localBlockPath;}
   public String getBlockPath() {return localBlockPath;}
-  
+
   /**
   /**
    * @return the Block
    * @return the Block
    */
    */
   public ExtendedBlock getBlock() { return block;}
   public ExtendedBlock getBlock() { return block;}
-  
+
   /**
   /**
    * Get the Block metadata file.
    * Get the Block metadata file.
    * @return 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.
    * @return a list of {@link StorageType}s for storing the replicas of a block.
    */
    */
   public List<StorageType> chooseStorageTypes(final short replication) {
   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;
     int i = 0, j = 0;
 
 
     // Do not return transient storage types. We will not have accurate
     // 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 Iterable<StorageType> chosen,
       final EnumSet<StorageType> unavailables,
       final EnumSet<StorageType> unavailables,
       final boolean isNewBlock) {
       final boolean isNewBlock) {
-    final List<StorageType> excess = new LinkedList<StorageType>();
+    final List<StorageType> excess = new LinkedList<>();
     final List<StorageType> storageTypes = chooseStorageTypes(
     final List<StorageType> storageTypes = chooseStorageTypes(
         replication, chosen, excess);
         replication, chosen, excess);
     final int expectedSize = storageTypes.size() - excess.size();
     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--) {
     for(int i = storageTypes.size() - 1; i >= 0; i--) {
       // replace/remove unavailable storage types.
       // replace/remove unavailable storage types.
       final StorageType t = storageTypes.get(i);
       final StorageType t = storageTypes.get(i);
@@ -195,7 +195,7 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
   public List<StorageType> chooseExcess(final short replication,
   public List<StorageType> chooseExcess(final short replication,
       final Iterable<StorageType> chosen) {
       final Iterable<StorageType> chosen) {
     final List<StorageType> types = chooseStorageTypes(replication);
     final List<StorageType> types = chooseStorageTypes(replication);
-    final List<StorageType> excess = new LinkedList<StorageType>();
+    final List<StorageType> excess = new LinkedList<>();
     diff(types, chosen, excess);
     diff(types, chosen, excess);
     return 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() {
   public CacheDirectiveStats getStats() {
     return stats;
     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.
      * Builds a new CacheDirectiveInfo populated with the set properties.
-     * 
+     *
      * @return New CacheDirectiveInfo.
      * @return New CacheDirectiveInfo.
      */
      */
     public CacheDirectiveInfo build() {
     public CacheDirectiveInfo build() {
@@ -73,7 +73,7 @@ public class CacheDirectiveInfo {
 
 
     /**
     /**
      * Sets the id used in this request.
      * Sets the id used in this request.
-     * 
+     *
      * @param id The id used in this request.
      * @param id The id used in this request.
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -84,7 +84,7 @@ public class CacheDirectiveInfo {
 
 
     /**
     /**
      * Sets the path used in this request.
      * Sets the path used in this request.
-     * 
+     *
      * @param path The path used in this request.
      * @param path The path used in this request.
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -95,7 +95,7 @@ public class CacheDirectiveInfo {
 
 
     /**
     /**
      * Sets the replication used in this request.
      * Sets the replication used in this request.
-     * 
+     *
      * @param replication The replication used in this request.
      * @param replication The replication used in this request.
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -106,7 +106,7 @@ public class CacheDirectiveInfo {
 
 
     /**
     /**
      * Sets the pool used in this request.
      * Sets the pool used in this request.
-     * 
+     *
      * @param pool The pool used in this request.
      * @param pool The pool used in this request.
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -119,7 +119,7 @@ public class CacheDirectiveInfo {
      * Sets when the CacheDirective should expire. A
      * Sets when the CacheDirective should expire. A
      * {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
      * {@link CacheDirectiveInfo.Expiration} can specify either an absolute or
      * relative expiration time.
      * relative expiration time.
-     * 
+     *
      * @param expiration when this CacheDirective should expire
      * @param expiration when this CacheDirective should expire
      * @return This builder, for call chaining
      * @return This builder, for call chaining
      */
      */
@@ -156,7 +156,7 @@ public class CacheDirectiveInfo {
      * <p>
      * <p>
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * expires.
      * expires.
-     * 
+     *
      * @param ms how long until the CacheDirective expires, in milliseconds
      * @param ms how long until the CacheDirective expires, in milliseconds
      * @return A relative Expiration
      * @return A relative Expiration
      */
      */
@@ -169,7 +169,7 @@ public class CacheDirectiveInfo {
      * <p>
      * <p>
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * expires.
      * expires.
-     * 
+     *
      * @param date when the CacheDirective expires
      * @param date when the CacheDirective expires
      * @return An absolute Expiration
      * @return An absolute Expiration
      */
      */
@@ -182,7 +182,7 @@ public class CacheDirectiveInfo {
      * <p>
      * <p>
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * Use {@link Expiration#NEVER} to indicate an Expiration that never
      * expires.
      * expires.
-     * 
+     *
      * @param ms when the CacheDirective expires, in milliseconds since the Unix
      * @param ms when the CacheDirective expires, in milliseconds since the Unix
      *          epoch.
      *          epoch.
      * @return An absolute Expiration
      * @return An absolute Expiration
@@ -350,9 +350,8 @@ public class CacheDirectiveInfo {
     }
     }
     if (expiration != null) {
     if (expiration != null) {
       builder.append(prefix).append("expiration: ").append(expiration);
       builder.append(prefix).append("expiration: ").append(expiration);
-      prefix = ", ";
     }
     }
     builder.append("}");
     builder.append("}");
     return builder.toString();
     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
   @Override
   public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
   public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
       throws IOException {
       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);
       entries = namenode.listCacheDirectives(prevKey, filter);
     } catch (IOException e) {
     } catch (IOException e) {
       if (e.getMessage().contains("Filtering by ID is unsupported")) {
       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
         // This is somewhat brittle, since it depends on directives being
         // returned in order of ascending ID.
         // returned in order of ascending ID.
         entries = namenode.listCacheDirectives(id - 1, filter);
         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);
           CacheDirectiveEntry entry = entries.get(i);
-          if (entry.getInfo().getId().equals((Long)id)) {
+          if (entry.getInfo().getId().equals(id)) {
             return new SingleEntry(entry);
             return new SingleEntry(entry);
           }
           }
         }
         }
@@ -115,8 +114,6 @@ public class CacheDirectiveIterator
             "Did not find requested id " + id);
             "Did not find requested id " + id);
       }
       }
       throw e;
       throw e;
-    } finally {
-      scope.close();
     }
     }
     Preconditions.checkNotNull(entries);
     Preconditions.checkNotNull(entries);
     return 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.
      * Builds a new CacheDirectiveStats populated with the set properties.
-     * 
+     *
      * @return New CacheDirectiveStats.
      * @return New CacheDirectiveStats.
      */
      */
     public CacheDirectiveStats build() {
     public CacheDirectiveStats build() {
@@ -51,7 +51,7 @@ public class CacheDirectiveStats {
 
 
     /**
     /**
      * Sets the bytes needed by this directive.
      * Sets the bytes needed by this directive.
-     * 
+     *
      * @param bytesNeeded The bytes needed.
      * @param bytesNeeded The bytes needed.
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -62,7 +62,7 @@ public class CacheDirectiveStats {
 
 
     /**
     /**
      * Sets the bytes cached by this directive.
      * Sets the bytes cached by this directive.
-     * 
+     *
      * @param bytesCached The bytes cached.
      * @param bytesCached The bytes cached.
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -83,7 +83,7 @@ public class CacheDirectiveStats {
 
 
     /**
     /**
      * Sets the files cached by this directive.
      * Sets the files cached by this directive.
-     * 
+     *
      * @param filesCached The number of files cached.
      * @param filesCached The number of files cached.
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -94,7 +94,7 @@ public class CacheDirectiveStats {
 
 
     /**
     /**
      * Sets whether this directive has expired.
      * Sets whether this directive has expired.
-     * 
+     *
      * @param hasExpired if this directive has expired
      * @param hasExpired if this directive has expired
      * @return This builder, for call chaining.
      * @return This builder, for call chaining.
      */
      */
@@ -156,14 +156,10 @@ public class CacheDirectiveStats {
 
 
   @Override
   @Override
   public String toString() {
   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
    * Set the maximum relative expiration of directives of this pool in
    * milliseconds.
    * milliseconds.
-   * 
+   *
    * @param ms in milliseconds
    * @param ms in milliseconds
    * @return This builder, for call chaining.
    * @return This builder, for call chaining.
    */
    */
@@ -155,17 +155,15 @@ public class CachePoolInfo {
   }
   }
 
 
   public String toString() {
   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
   @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
     if (o == null) { return false; }
     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
   @Override
   public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
   public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
       throws IOException {
       throws IOException {
-    TraceScope scope = tracer.newScope("listCachePools");
-    try {
+    try (TraceScope ignored = tracer.newScope("listCachePools")) {
       return namenode.listCachePools(prevKey);
       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,
       return new CachePoolStats(bytesNeeded, bytesCached, bytesOverlimit,
           filesNeeded, filesCached);
           filesNeeded, filesCached);
     }
     }
-  };
+  }
 
 
   private final long bytesNeeded;
   private final long bytesNeeded;
   private final long bytesCached;
   private final long bytesCached;
@@ -104,12 +104,10 @@ public class CachePoolStats {
   }
   }
 
 
   public String toString() {
   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 {
 public interface ClientDatanodeProtocol {
   /**
   /**
    * Until version 9, this class ClientDatanodeProtocol served as both
    * 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.
    * 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.
    * 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
    * change both this class and ALSO related protocol buffer
    * wire protocol definition in ClientDatanodeProtocol.proto.
    * 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
    * .../org/apache/hadoop/hdfs/protocolPB/overview.html
-   * 
+   *
    * The log of historical changes can be retrieved from the svn).
    * The log of historical changes can be retrieved from the svn).
    * 9: Added deleteBlockPool method
    * 9: Added deleteBlockPool method
-   * 
+   *
    * 9 is the last version id when this class was used for protocols
    * 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. */
   /** Return the visible length of a replica. */
   long getReplicaVisibleLength(ExtendedBlock b) throws IOException;
   long getReplicaVisibleLength(ExtendedBlock b) throws IOException;
-  
+
   /**
   /**
    * Refresh the list of federated namenodes from updated configuration
    * Refresh the list of federated namenodes from updated configuration
    * Adds new namenodes and stops the deleted namenodes.
    * Adds new namenodes and stops the deleted namenodes.
-   * 
+   *
    * @throws IOException on error
    * @throws IOException on error
    **/
    **/
   void refreshNamenodes() throws IOException;
   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
    * 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.
    * it is empty, otherwise it is deleted along with its contents.
-   * 
+   *
    * @param bpid Blockpool id to be deleted.
    * @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.
    *          deleted along with its contents.
    * @throws IOException
    * @throws IOException
    */
    */
   void deleteBlockPool(String bpid, boolean force) throws IOException;
   void deleteBlockPool(String bpid, boolean force) throws IOException;
-  
+
   /**
   /**
    * Retrieves the path names of the block file and metadata file stored on the
    * Retrieves the path names of the block file and metadata file stored on the
    * local file system.
    * local file system.
-   * 
+   *
    * In order for this method to work, one of the following should be satisfied:
    * In order for this method to work, one of the following should be satisfied:
    * <ul>
    * <ul>
    * <li>
    * <li>
@@ -98,7 +98,7 @@ public interface ClientDatanodeProtocol {
    * When security is enabled, kerberos authentication must be used to connect
    * When security is enabled, kerberos authentication must be used to connect
    * to the datanode.</li>
    * to the datanode.</li>
    * </ul>
    * </ul>
-   * 
+   *
    * @param block
    * @param block
    *          the specified block on the local datanode
    *          the specified block on the local datanode
    * @param token
    * @param token
@@ -109,7 +109,7 @@ public interface ClientDatanodeProtocol {
    */
    */
   BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
   BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block,
       Token<BlockTokenIdentifier> token) throws IOException;
       Token<BlockTokenIdentifier> token) throws IOException;
-  
+
   /**
   /**
    * Shuts down a datanode.
    * Shuts down a datanode.
    *
    *
@@ -117,9 +117,9 @@ public interface ClientDatanodeProtocol {
    *          down. The work includes advising clients to wait and saving
    *          down. The work includes advising clients to wait and saving
    *          certain states for quick restart. This should only be used when
    *          certain states for quick restart. This should only be used when
    *          the stored data will remain the same during upgrade/restart.
    *          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
    * 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.Text;
 import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.Idempotent;
 import org.apache.hadoop.io.retry.Idempotent;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.KerberosInfo;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenInfo;
 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();
     String msg = super.getMessage();
     if (msg == null) {
     if (msg == null) {
       return "The DiskSpace quota" + (pathName==null?"": " of " + pathName)
       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 {
     } else {
       return msg;
       return msg;
     }
     }

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

@@ -235,14 +235,10 @@ public class DatanodeID implements Comparable<DatanodeID> {
 
 
   @Override
   @Override
   public boolean equals(Object to) {
   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
   @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 int xceiverCount;
   private String location = NetworkTopology.DEFAULT_RACK;
   private String location = NetworkTopology.DEFAULT_RACK;
   private String softwareVersion;
   private String softwareVersion;
-  private List<String> dependentHostNames = new LinkedList<String>();
+  private List<String> dependentHostNames = new LinkedList<>();
   private String upgradeDomain;
   private String upgradeDomain;
 
 
   // Datanode administrative states
   // Datanode administrative states
@@ -360,18 +360,18 @@ public class DatanodeInfo extends DatanodeID implements Node {
     float cacheRemainingPercent = getCacheRemainingPercent();
     float cacheRemainingPercent = getCacheRemainingPercent();
     String lookupName = NetUtils.getHostNameOfIP(getName());
     String lookupName = NetUtils.getHostNameOfIP(getName());
 
 
-    buffer.append("Name: "+ getName());
+    buffer.append("Name: ").append(getName());
     if (lookupName != null) {
     if (lookupName != null) {
-      buffer.append(" (" + lookupName + ")");
+      buffer.append(" (").append(lookupName).append(")");
     }
     }
     buffer.append("\n");
     buffer.append("\n");
-    buffer.append("Hostname: " + getHostName() + "\n");
+    buffer.append("Hostname: ").append(getHostName()).append("\n");
 
 
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
-      buffer.append("Rack: "+location+"\n");
+      buffer.append("Rack: ").append(location).append("\n");
     }
     }
     if (upgradeDomain != null) {
     if (upgradeDomain != null) {
-      buffer.append("Upgrade domain: "+ upgradeDomain +"\n");
+      buffer.append("Upgrade domain: ").append(upgradeDomain).append("\n");
     }
     }
     buffer.append("Decommission Status : ");
     buffer.append("Decommission Status : ");
     if (isDecommissioned()) {
     if (isDecommissioned()) {
@@ -381,19 +381,30 @@ public class DatanodeInfo extends DatanodeID implements Node {
     } else {
     } else {
       buffer.append("Normal\n");
       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();
     return buffer.toString();
   }
   }
 
 
@@ -410,10 +421,10 @@ public class DatanodeInfo extends DatanodeID implements Node {
     float cacheUsedPercent = getCacheUsedPercent();
     float cacheUsedPercent = getCacheUsedPercent();
     buffer.append(getName());
     buffer.append(getName());
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
     if (!NetworkTopology.DEFAULT_RACK.equals(location)) {
-      buffer.append(" "+location);
+      buffer.append(" ").append(location);
     }
     }
     if (upgradeDomain != null) {
     if (upgradeDomain != null) {
-      buffer.append(" " + upgradeDomain);
+      buffer.append(" ").append(upgradeDomain);
     }
     }
     if (isDecommissioned()) {
     if (isDecommissioned()) {
       buffer.append(" DD");
       buffer.append(" DD");
@@ -422,15 +433,21 @@ public class DatanodeInfo extends DatanodeID implements Node {
     } else {
     } else {
       buffer.append(" IN");
       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();
     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.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
@@ -29,7 +28,7 @@ public class DatanodeInfoWithStorage extends DatanodeInfo {
   private final StorageType storageType;
   private final StorageType storageType;
 
 
   public DatanodeInfoWithStorage(DatanodeInfo from, String storageID,
   public DatanodeInfoWithStorage(DatanodeInfo from, String storageID,
-                                 StorageType storageType) {
+      StorageType storageType) {
     super(from);
     super(from);
     this.storageID = storageID;
     this.storageID = storageID;
     this.storageType = storageType;
     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. */
   /** A formatted string for printing the status of the DataNode. */
   public String getDatanodeLocalReport() {
   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;
   private final Tracer tracer;
 
 
   public EncryptionZoneIterator(ClientProtocol namenode, Tracer tracer) {
   public EncryptionZoneIterator(ClientProtocol namenode, Tracer tracer) {
-    super(Long.valueOf(0));
+    super((long) 0);
     this.namenode = namenode;
     this.namenode = namenode;
     this.tracer = tracer;
     this.tracer = tracer;
   }
   }
@@ -47,11 +47,8 @@ public class EncryptionZoneIterator
   @Override
   @Override
   public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
   public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
       throws IOException {
       throws IOException {
-    TraceScope scope = tracer.newScope("listEncryptionZones");
-    try {
+    try (TraceScope ignored = tracer.newScope("listEncryptionZones")) {
       return namenode.listEncryptionZones(prevId);
       return namenode.listEncryptionZones(prevId);
-    } finally {
-      scope.close();
     }
     }
   }
   }
 
 

+ 6 - 13
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java

@@ -19,8 +19,6 @@ package org.apache.hadoop.hdfs.protocol;
 
 
 import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.ECSchema;
 
 
-import java.util.Map;
-
 /**
 /**
  * A policy about how to write/read/code an erasure coding file.
  * A policy about how to write/read/code an erasure coding file.
  */
  */
@@ -66,11 +64,9 @@ public final class ErasureCodingPolicy {
     }
     }
     ErasureCodingPolicy that = (ErasureCodingPolicy) o;
     ErasureCodingPolicy that = (ErasureCodingPolicy) o;
 
 
-    if (that.getName().equals(name) && that.getCellSize() == cellSize
-        && that.getSchema().equals(schema)) {
-      return true;
-    }
-    return false;
+    return that.getName().equals(name) &&
+        that.getCellSize() == cellSize &&
+        that.getSchema().equals(schema);
   }
   }
 
 
   @Override
   @Override
@@ -83,11 +79,8 @@ public final class ErasureCodingPolicy {
 
 
   @Override
   @Override
   public String toString() {
   public String toString() {
-    StringBuilder sb = new StringBuilder("ErasureCodingPolicy=[");
-    sb.append("Name=" + name + ", ");
-    sb.append("Schema=[" + schema.toString() + "], ");
-    sb.append("CellSize=" + cellSize + " ");
-    sb.append("]");
-    return sb.toString();
+    return "ErasureCodingPolicy=[" + "Name=" + name + ", "
+        + "Schema=[" + schema.toString() + "], "
+        + "CellSize=" + cellSize + " " + "]";
   }
   }
 }
 }

+ 3 - 2
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
 @InterfaceStability.Evolving
 public class HdfsFileStatus {
 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 byte[] symlink; // symlink target encoded in java UTF8 or null
   private final long length;
   private final long length;
   private final boolean isdir;
   private final boolean isdir;
@@ -49,7 +50,7 @@ public class HdfsFileStatus {
   private final FileEncryptionInfo feInfo;
   private final FileEncryptionInfo feInfo;
 
 
   private final ErasureCodingPolicy ecPolicy;
   private final ErasureCodingPolicy ecPolicy;
-  
+
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   // Used by dir, not including dot and dotdot. Always zero for a regular file.
   private final int childrenNum;
   private final int childrenNum;
   private final byte storagePolicy;
   private final byte storagePolicy;

+ 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.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 
 
-/** 
+/**
  * Interface that represents the over the wire information
  * Interface that represents the over the wire information
  * including block locations for a file.
  * including block locations for a file.
  */
  */
@@ -38,7 +38,7 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
 
 
   /**
   /**
    * Constructor
    * Constructor
-   * 
+   *
    * @param length size
    * @param length size
    * @param isdir if this is directory
    * @param isdir if this is directory
    * @param block_replication the file's replication factor
    * @param block_replication the file's replication factor
@@ -49,7 +49,7 @@ public class HdfsLocatedFileStatus extends HdfsFileStatus {
    * @param owner owner
    * @param owner owner
    * @param group group
    * @param group group
    * @param symlink symbolic link
    * @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 fileId the file id
    * @param locations block locations
    * @param locations block locations
    * @param feInfo file encryption info
    * @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;
   private final HdfsFileStatus fileStatus;
 
 
-  public LastBlockWithStatus(LocatedBlock lastBlock, HdfsFileStatus fileStatus) {
+  public LastBlockWithStatus(LocatedBlock lastBlock,
+      HdfsFileStatus fileStatus) {
     this.lastBlock = lastBlock;
     this.lastBlock = lastBlock;
     this.fileStatus = fileStatus;
     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,
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
-                      String[] storageIDs, StorageType[] storageTypes) {
+      String[] storageIDs, StorageType[] storageTypes) {
     this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
     this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
   }
   }
 
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
   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.b = b;
     this.offset = startOffset;
     this.offset = startOffset;
     this.corrupt = corrupt;
     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
 @InterfaceStability.Evolving
 public class LocatedBlocks {
 public class LocatedBlocks {
   private final long fileLength;
   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 boolean underConstruction;
   private final LocatedBlock lastLocatedBlock;
   private final LocatedBlock lastLocatedBlock;
   private final boolean isLastBlockComplete;
   private final boolean isLastBlockComplete;
@@ -134,22 +135,22 @@ public class LocatedBlocks {
     key.setStartOffset(offset);
     key.setStartOffset(offset);
     key.getBlock().setNumBytes(1);
     key.getBlock().setNumBytes(1);
     Comparator<LocatedBlock> comp =
     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);
     return Collections.binarySearch(blocks, key, comp);
   }
   }
 
 
@@ -187,14 +188,10 @@ public class LocatedBlocks {
 
 
   @Override
   @Override
   public String toString() {
   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 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java

@@ -39,6 +39,7 @@ public class LocatedStripedBlock extends LocatedBlock {
   private int[] blockIndices;
   private int[] blockIndices;
   private Token<BlockTokenIdentifier>[] blockTokens;
   private Token<BlockTokenIdentifier>[] blockTokens;
 
 
+  @SuppressWarnings({"unchecked"})
   public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs,
   public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs,
       String[] storageIDs, StorageType[] storageTypes, int[] indices,
       String[] storageIDs, StorageType[] storageTypes, int[] indices,
       long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) {
       long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) {

+ 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();
     String msg = super.getMessage();
     if (msg == null) {
     if (msg == null) {
       msg = "The NameSpace quota (directories and files)" +
       msg = "The NameSpace quota (directories and files)" +
-      (pathName==null?"":(" of directory " + pathName)) +
+          (pathName==null?"":(" of directory " + pathName)) +
           " is exceeded: quota=" + quota + " file count=" + count;
           " is exceeded: quota=" + quota + " file count=" + count;
 
 
       if (prefix != null) {
       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
    * Finalize the upgrade if not already finalized
-   * @param finalizeTime
    */
    */
   public void finalize(long finalizeTime) {
   public void finalize(long finalizeTime) {
     if (finalizeTime != 0) {
     if (finalizeTime != 0) {
@@ -99,8 +98,11 @@ public class RollingUpgradeInfo extends RollingUpgradeStatus {
   @Override
   @Override
   public String toString() {
   public String toString() {
     return super.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) {
   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;
 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
  * 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
  * its current state. Instead of capturing all the details of the diff, this
  * class only lists where the changes happened and their types.
  * class only lists where the changes happened and their types.
@@ -42,21 +42,21 @@ public class SnapshotDiffReport {
    * DELETE, and RENAME respectively.
    * DELETE, and RENAME respectively.
    */
    */
   public enum DiffType {
   public enum DiffType {
-    CREATE("+"),     
-    MODIFY("M"),    
-    DELETE("-"), 
+    CREATE("+"),
+    MODIFY("M"),
+    DELETE("-"),
     RENAME("R");
     RENAME("R");
-    
+
     private final String label;
     private final String label;
-    
-    private DiffType(String label) {
+
+    DiffType(String label) {
       this.label = label;
       this.label = label;
     }
     }
-    
+
     public String getLabel() {
     public String getLabel() {
       return label;
       return label;
     }
     }
-    
+
     public static DiffType getTypeFromLabel(String label) {
     public static DiffType getTypeFromLabel(String label) {
       if (label.equals(CREATE.getLabel())) {
       if (label.equals(CREATE.getLabel())) {
         return CREATE;
         return CREATE;
@@ -69,8 +69,8 @@ public class SnapshotDiffReport {
       }
       }
       return null;
       return null;
     }
     }
-  };
-  
+  }
+
   /**
   /**
    * Representing the full path and diff type of a file/directory where changes
    * Representing the full path and diff type of a file/directory where changes
    * have happened.
    * have happened.
@@ -98,7 +98,7 @@ public class SnapshotDiffReport {
       this.sourcePath = sourcePath;
       this.sourcePath = sourcePath;
       this.targetPath = targetPath;
       this.targetPath = targetPath;
     }
     }
-    
+
     public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
     public DiffReportEntry(DiffType type, byte[][] sourcePathComponents,
         byte[][] targetPathComponents) {
         byte[][] targetPathComponents) {
       this.type = type;
       this.type = type;
@@ -106,7 +106,7 @@ public class SnapshotDiffReport {
       this.targetPath = targetPathComponents == null ? null : DFSUtilClient
       this.targetPath = targetPathComponents == null ? null : DFSUtilClient
           .byteArray2bytes(targetPathComponents);
           .byteArray2bytes(targetPathComponents);
     }
     }
-    
+
     @Override
     @Override
     public String toString() {
     public String toString() {
       String str = type.getLabel() + "\t" + getPathString(sourcePath);
       String str = type.getLabel() + "\t" + getPathString(sourcePath);
@@ -115,7 +115,7 @@ public class SnapshotDiffReport {
       }
       }
       return str;
       return str;
     }
     }
-    
+
     public DiffType getType() {
     public DiffType getType() {
       return type;
       return type;
     }
     }
@@ -141,7 +141,7 @@ public class SnapshotDiffReport {
     public boolean equals(Object other) {
     public boolean equals(Object other) {
       if (this == other) {
       if (this == other) {
         return true;
         return true;
-      } 
+      }
       if (other != null && other instanceof DiffReportEntry) {
       if (other != null && other instanceof DiffReportEntry) {
         DiffReportEntry entry = (DiffReportEntry) other;
         DiffReportEntry entry = (DiffReportEntry) other;
         return type.equals(entry.getType())
         return type.equals(entry.getType())
@@ -150,25 +150,25 @@ public class SnapshotDiffReport {
       }
       }
       return false;
       return false;
     }
     }
-    
+
     @Override
     @Override
     public int hashCode() {
     public int hashCode() {
       return Objects.hashCode(getSourcePath(), getTargetPath());
       return Objects.hashCode(getSourcePath(), getTargetPath());
     }
     }
   }
   }
-  
+
   /** snapshot root full path */
   /** snapshot root full path */
   private final String snapshotRoot;
   private final String snapshotRoot;
 
 
   /** start point of the diff */
   /** start point of the diff */
   private final String fromSnapshot;
   private final String fromSnapshot;
-  
+
   /** end point of the diff */
   /** end point of the diff */
   private final String toSnapshot;
   private final String toSnapshot;
-  
+
   /** list of diff */
   /** list of diff */
   private final List<DiffReportEntry> diffList;
   private final List<DiffReportEntry> diffList;
-  
+
   public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
   public SnapshotDiffReport(String snapshotRoot, String fromSnapshot,
       String toSnapshot, List<DiffReportEntry> entryList) {
       String toSnapshot, List<DiffReportEntry> entryList) {
     this.snapshotRoot = snapshotRoot;
     this.snapshotRoot = snapshotRoot;
@@ -177,7 +177,7 @@ public class SnapshotDiffReport {
     this.diffList = entryList != null ? entryList : Collections
     this.diffList = entryList != null ? entryList : Collections
         .<DiffReportEntry> emptyList();
         .<DiffReportEntry> emptyList();
   }
   }
-  
+
   /** @return {@link #snapshotRoot}*/
   /** @return {@link #snapshotRoot}*/
   public String getSnapshotRoot() {
   public String getSnapshotRoot() {
     return snapshotRoot;
     return snapshotRoot;
@@ -192,23 +192,24 @@ public class SnapshotDiffReport {
   public String getLaterSnapshotName() {
   public String getLaterSnapshotName() {
     return toSnapshot;
     return toSnapshot;
   }
   }
-  
+
   /** @return {@link #diffList} */
   /** @return {@link #diffList} */
   public List<DiffReportEntry> getDiffList() {
   public List<DiffReportEntry> getDiffList() {
     return diffList;
     return diffList;
   }
   }
-  
+
   @Override
   @Override
   public String toString() {
   public String toString() {
     StringBuilder str = new StringBuilder();
     StringBuilder str = new StringBuilder();
-    String from = fromSnapshot == null || fromSnapshot.isEmpty() ? 
+    String from = fromSnapshot == null || fromSnapshot.isEmpty() ?
         "current directory" : "snapshot " + fromSnapshot;
         "current directory" : "snapshot " + fromSnapshot;
     String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
     String to = toSnapshot == null || toSnapshot.isEmpty() ? "current directory"
         : "snapshot " + toSnapshot;
         : "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) {
     for (DiffReportEntry entry : diffList) {
-      str.append(entry.toString() + LINE_SEPARATOR);
+      str.append(entry.toString()).append(LINE_SEPARATOR);
     }
     }
     return str.toString();
     return str.toString();
   }
   }

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

@@ -31,37 +31,39 @@ import org.apache.hadoop.hdfs.DFSUtilClient;
  */
  */
 public class SnapshottableDirectoryStatus {
 public class SnapshottableDirectoryStatus {
   /** Compare the statuses by full paths. */
   /** 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());
               right.dirStatus.getLocalNameInBytes());
-    }
-  };
+        }
+      };
 
 
   /** Basic information of the snapshottable directory */
   /** Basic information of the snapshottable directory */
   private final HdfsFileStatus dirStatus;
   private final HdfsFileStatus dirStatus;
-  
+
   /** Number of snapshots that have been taken*/
   /** Number of snapshots that have been taken*/
   private final int snapshotNumber;
   private final int snapshotNumber;
-  
+
   /** Number of snapshots allowed. */
   /** Number of snapshots allowed. */
   private final int snapshotQuota;
   private final int snapshotQuota;
-  
+
   /** Full path of the parent. */
   /** Full path of the parent. */
   private final byte[] parentFullPath;
   private final byte[] parentFullPath;
-  
+
   public SnapshottableDirectoryStatus(long modification_time, long access_time,
   public SnapshottableDirectoryStatus(long modification_time, long access_time,
       FsPermission permission, String owner, String group, byte[] localName,
       FsPermission permission, String owner, String group, byte[] localName,
       long inodeId, int childrenNum,
       long inodeId, int childrenNum,
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
       int snapshotNumber, int snapshotQuota, byte[] parentFullPath) {
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
     this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time,
         access_time, permission, owner, group, null, localName, inodeId,
         access_time, permission, owner, group, null, localName, inodeId,
-        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null);
+        childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
+        null);
     this.snapshotNumber = snapshotNumber;
     this.snapshotNumber = snapshotNumber;
     this.snapshotQuota = snapshotQuota;
     this.snapshotQuota = snapshotQuota;
     this.parentFullPath = parentFullPath;
     this.parentFullPath = parentFullPath;
@@ -80,7 +82,7 @@ public class SnapshottableDirectoryStatus {
   public int getSnapshotQuota() {
   public int getSnapshotQuota() {
     return snapshotQuota;
     return snapshotQuota;
   }
   }
-  
+
   /**
   /**
    * @return Full path of the parent
    * @return Full path of the parent
    */
    */
@@ -94,13 +96,13 @@ public class SnapshottableDirectoryStatus {
   public HdfsFileStatus getDirStatus() {
   public HdfsFileStatus getDirStatus() {
     return dirStatus;
     return dirStatus;
   }
   }
-  
+
   /**
   /**
    * @return Full path of the file
    * @return Full path of the file
    */
    */
   public Path getFullPath() {
   public Path getFullPath() {
-    String parentFullPathStr = 
-        (parentFullPath == null || parentFullPath.length == 0) ? 
+    String parentFullPathStr =
+        (parentFullPath == null || parentFullPath.length == 0) ?
             null : DFSUtilClient.bytes2String(parentFullPath);
             null : DFSUtilClient.bytes2String(parentFullPath);
     if (parentFullPathStr == null
     if (parentFullPathStr == null
         && dirStatus.getLocalNameInBytes().length == 0) {
         && dirStatus.getLocalNameInBytes().length == 0) {
@@ -111,13 +113,13 @@ public class SnapshottableDirectoryStatus {
           : new Path(parentFullPathStr, dirStatus.getLocalName());
           : new Path(parentFullPathStr, dirStatus.getLocalName());
     }
     }
   }
   }
-  
+
   /**
   /**
    * Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
    * Print a list of {@link SnapshottableDirectoryStatus} out to a given stream.
    * @param stats The list of {@link SnapshottableDirectoryStatus}
    * @param stats The list of {@link SnapshottableDirectoryStatus}
    * @param out The given stream for printing.
    * @param out The given stream for printing.
    */
    */
-  public static void print(SnapshottableDirectoryStatus[] stats, 
+  public static void print(SnapshottableDirectoryStatus[] stats,
       PrintStream out) {
       PrintStream out) {
     if (stats == null || stats.length == 0) {
     if (stats == null || stats.length == 0) {
       out.println();
       out.println();
@@ -133,30 +135,28 @@ public class SnapshottableDirectoryStatus {
       maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
       maxSnapshotNum = maxLength(maxSnapshotNum, status.snapshotNumber);
       maxSnapshotQuota = maxLength(maxSnapshotQuota, status.snapshotQuota);
       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");
     SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm");
-         
+
     for (SnapshottableDirectoryStatus status : stats) {
     for (SnapshottableDirectoryStatus status : stats) {
-      String line = String.format(lineFormat, "d", 
+      String line = String.format(lineFormat, "d",
           status.dirStatus.getPermission(),
           status.dirStatus.getPermission(),
           status.dirStatus.getReplication(),
           status.dirStatus.getReplication(),
           status.dirStatus.getOwner(),
           status.dirStatus.getOwner(),
           status.dirStatus.getGroup(),
           status.dirStatus.getGroup(),
           String.valueOf(status.dirStatus.getLen()),
           String.valueOf(status.dirStatus.getLen()),
           dateFormat.format(new Date(status.dirStatus.getModificationTime())),
           dateFormat.format(new Date(status.dirStatus.getModificationTime())),
-          status.snapshotNumber, status.snapshotQuota, 
+          status.snapshotNumber, status.snapshotQuota,
           status.getFullPath().toString()
           status.getFullPath().toString()
       );
       );
       out.println(line);
       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;
 package org.apache.hadoop.hdfs.protocol;
 
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 
 
-/** 
+/**
  * Thrown when a symbolic link is encountered in a path.
  * Thrown when a symbolic link is encountered in a path.
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
@@ -43,7 +41,7 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
   public UnresolvedPathException(String msg) {
   public UnresolvedPathException(String msg) {
     super(msg);
     super(msg);
   }
   }
-  
+
   public UnresolvedPathException(String path, String preceding,
   public UnresolvedPathException(String path, String preceding,
       String remainder, String linkTarget) {
       String remainder, String linkTarget) {
     this.path = path;
     this.path = path;
@@ -55,7 +53,7 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
   /**
   /**
    * Return a path with the link resolved with the target.
    * 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
     // If the path is absolute we cam throw out the preceding part and
     // just append the remainder to the target, otherwise append each
     // just append the remainder to the target, otherwise append each
     // piece to resolve the link in path.
     // piece to resolve the link in path.
@@ -76,12 +74,6 @@ public final class UnresolvedPathException extends UnresolvedLinkException {
     if (msg != null) {
     if (msg != null) {
       return msg;
       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
 @InterfaceStability.Evolving
 public enum BlockConstructionStage {
 public enum BlockConstructionStage {
   /** The enumerates are always listed as regular stage followed by the
   /** The enumerates are always listed as regular stage followed by the
-   * recovery stage. 
+   * recovery stage.
    * Changing this order will make getRecoveryStage not working.
    * Changing this order will make getRecoveryStage not working.
    */
    */
   // pipeline set up for block append
   // pipeline set up for block append
@@ -46,9 +46,9 @@ public enum BlockConstructionStage {
   TRANSFER_RBW,
   TRANSFER_RBW,
   // transfer Finalized for adding datanodes
   // transfer Finalized for adding datanodes
   TRANSFER_FINALIZED;
   TRANSFER_FINALIZED;
-  
+
   final static private byte RECOVERY_BIT = (byte)1;
   final static private byte RECOVERY_BIT = (byte)1;
-  
+
   /**
   /**
    * get the recovery stage of this stage
    * get the recovery stage of this stage
    */
    */
@@ -59,4 +59,4 @@ public enum BlockConstructionStage {
       return values()[ordinal()|RECOVERY_BIT];
       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) {
   public static DataChecksum fromProto(ChecksumProto proto) {
-    if (proto == null) return null;
+    if (proto == null) {
+      return null;
+    }
 
 
     int bytesPerChecksum = proto.getBytesPerChecksum();
     int bytesPerChecksum = proto.getBytesPerChecksum();
     DataChecksum.Type type = PBHelperClient.convert(proto.getType());
     DataChecksum.Type type = PBHelperClient.convert(proto.getType());
@@ -74,19 +76,17 @@ public abstract class DataTransferProtoUtil {
 
 
   static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk,
   static ClientOperationHeaderProto buildClientHeader(ExtendedBlock blk,
       String client, Token<BlockTokenIdentifier> blockToken) {
       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,
   static BaseHeaderProto buildBaseHeader(ExtendedBlock blk,
       Token<BlockTokenIdentifier> blockToken) {
       Token<BlockTokenIdentifier> blockToken) {
     BaseHeaderProto.Builder builder =  BaseHeaderProto.newBuilder()
     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();
     SpanId spanId = Tracer.getCurrentSpanId();
     if (spanId.isValid()) {
     if (spanId.isValid()) {
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder()
       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
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public interface DataTransferProtocol {
 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
   /** Version for data transfers between clients and datanodes
    * This should change when serialization of DatanodeInfo, not just
    * 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:
    * Version 28:
    *    Declare methods in DataTransferProtocol interface.
    *    Declare methods in DataTransferProtocol interface.
    */
    */
-  public static final int DATA_TRANSFER_VERSION = 28;
+  int DATA_TRANSFER_VERSION = 28;
 
 
-  /** 
+  /**
    * Read a block.
    * Read a block.
-   * 
+   *
    * @param blk the block being read.
    * @param blk the block being read.
    * @param blockToken security token for accessing the block.
    * @param blockToken security token for accessing the block.
    * @param clientName client's name.
    * @param clientName client's name.
@@ -63,7 +63,7 @@ public interface DataTransferProtocol {
    *        checksums
    *        checksums
    * @param cachingStrategy  The caching strategy to use.
    * @param cachingStrategy  The caching strategy to use.
    */
    */
-  public void readBlock(final ExtendedBlock blk,
+  void readBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final String clientName,
       final long blockOffset,
       final long blockOffset,
@@ -77,7 +77,7 @@ public interface DataTransferProtocol {
    * The other downstream datanodes are specified by the targets parameter.
    * The other downstream datanodes are specified by the targets parameter.
    * Note that the receiver {@link DatanodeInfo} is not required in the
    * Note that the receiver {@link DatanodeInfo} is not required in the
    * parameter list since the receiver datanode knows its info.  However, 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.
    * parameter since the receiver datanode may support multiple storage types.
    *
    *
    * @param blk the block being written.
    * @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 pinning whether to pin the block, so Balancer won't move it.
    * @param targetPinnings whether to pin the block on target datanode
    * @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 Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final String clientName,
       final DatanodeInfo[] targets,
       final DatanodeInfo[] targets,
-      final StorageType[] targetStorageTypes, 
+      final StorageType[] targetStorageTypes,
       final DatanodeInfo source,
       final DatanodeInfo source,
       final BlockConstructionStage stage,
       final BlockConstructionStage stage,
       final int pipelineSize,
       final int pipelineSize,
@@ -118,13 +118,13 @@ public interface DataTransferProtocol {
    * The block stage must be
    * The block stage must be
    * either {@link BlockConstructionStage#TRANSFER_RBW}
    * either {@link BlockConstructionStage#TRANSFER_RBW}
    * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
    * or {@link BlockConstructionStage#TRANSFER_FINALIZED}.
-   * 
+   *
    * @param blk the block being transferred.
    * @param blk the block being transferred.
    * @param blockToken security token for accessing the block.
    * @param blockToken security token for accessing the block.
    * @param clientName client's name.
    * @param clientName client's name.
    * @param targets target datanodes.
    * @param targets target datanodes.
    */
    */
-  public void transferBlock(final ExtendedBlock blk,
+  void transferBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final String clientName,
       final DatanodeInfo[] targets,
       final DatanodeInfo[] targets,
@@ -135,14 +135,14 @@ public interface DataTransferProtocol {
    *
    *
    * @param blk             The block to get file descriptors for.
    * @param blk             The block to get file descriptors for.
    * @param blockToken      Security token for accessing the block.
    * @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.
    *                          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.
    *                          can understand.
    * @param supportsReceiptVerification  True if the client supports
    * @param supportsReceiptVerification  True if the client supports
    *                          receipt verification.
    *                          receipt verification.
    */
    */
-  public void requestShortCircuitFds(final ExtendedBlock blk,
+  void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       final Token<BlockTokenIdentifier> blockToken,
       SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
       SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
         throws IOException;
         throws IOException;
@@ -152,51 +152,51 @@ public interface DataTransferProtocol {
    *
    *
    * @param slotId          SlotID used by the earlier file descriptors.
    * @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.
    * Request a short circuit shared memory area from a DataNode.
-   * 
+   *
    * @param clientName       The name of the client.
    * @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
    * Receive a block from a source datanode
    * and then notifies the namenode
    * and then notifies the namenode
    * to remove the copy from the original datanode.
    * to remove the copy from the original datanode.
    * Note that the source datanode and the original datanode can be different.
    * Note that the source datanode and the original datanode can be different.
    * It is used for balancing purpose.
    * It is used for balancing purpose.
-   * 
+   *
    * @param blk the block being replaced.
    * @param blk the block being replaced.
    * @param storageType the {@link StorageType} for storing the block.
    * @param storageType the {@link StorageType} for storing the block.
    * @param blockToken security token for accessing the block.
    * @param blockToken security token for accessing the block.
    * @param delHint the hint for deleting the block in the original datanode.
    * @param delHint the hint for deleting the block in the original datanode.
    * @param source the source datanode for receiving the block.
    * @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 Token<BlockTokenIdentifier> blockToken,
       final String delHint,
       final String delHint,
       final DatanodeInfo source) throws IOException;
       final DatanodeInfo source) throws IOException;
 
 
   /**
   /**
-   * Copy a block. 
+   * Copy a block.
    * It is used for balancing purpose.
    * It is used for balancing purpose.
-   * 
+   *
    * @param blk the block being copied.
    * @param blk the block being copied.
    * @param blockToken security token for accessing the block.
    * @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;
       final Token<BlockTokenIdentifier> blockToken) throws IOException;
 
 
   /**
   /**
    * Get block checksum (MD5 of CRC32).
    * Get block checksum (MD5 of CRC32).
-   * 
+   *
    * @param blk a block.
    * @param blk a block.
    * @param blockToken security token for accessing the block.
    * @param blockToken security token for accessing the block.
    * @throws IOException
    * @throws IOException
    */
    */
-  public void blockChecksum(final ExtendedBlock blk,
+  void blockChecksum(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken) throws IOException;
       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 class IOStreamPair {
   public final InputStream in;
   public final InputStream in;
   public final OutputStream out;
   public final OutputStream out;
-  
+
   public IOStreamPair(InputStream in, OutputStream out) {
   public IOStreamPair(InputStream in, OutputStream out) {
     this.in = in;
     this.in = in;
     this.out = out;
     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. */
   /** The code for this operation. */
   public final byte code;
   public final byte code;
-  
-  private Op(byte code) {
+
+  Op(byte code) {
     this.code = code;
     this.code = code;
   }
   }
-  
+
   private static final int FIRST_CODE = values()[0].code;
   private static final int FIRST_CODE = values()[0].code;
   /** Return the object represented by the code. */
   /** Return the object represented by the code. */
   private static Op valueOf(byte 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.
  * Header data for each packet that goes through the read/write pipelines.
  * Includes all of the information about the packet, excluding checksums and
  * Includes all of the information about the packet, excluding checksums and
  * actual data.
  * actual data.
- * 
+ *
  * This data includes:
  * This data includes:
  *  - the offset in bytes into the HDFS block of the data in this packet
  *  - the offset in bytes into the HDFS block of the data in this packet
  *  - the sequence number of this packet in the pipeline
  *  - the sequence number of this packet in the pipeline
  *  - whether or not this is the last packet in the pipeline
  *  - whether or not this is the last packet in the pipeline
  *  - the length of the data in this packet
  *  - the length of the data in this packet
  *  - whether or not this packet should be synced by the DNs.
  *  - whether or not this packet should be synced by the DNs.
- *  
+ *
  * When serialized, this header is written out as a protocol buffer, preceded
  * 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
  * by a 4-byte integer representing the full packet length, and a 2-byte short
  * representing the header length.
  * representing the header length.
@@ -51,8 +51,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
 public class PacketHeader {
 public class PacketHeader {
-  private static final int MAX_PROTO_SIZE = 
-    PacketHeaderProto.newBuilder()
+  private static final int MAX_PROTO_SIZE = PacketHeaderProto.newBuilder()
       .setOffsetInBlock(0)
       .setOffsetInBlock(0)
       .setSeqno(0)
       .setSeqno(0)
       .setLastPacketInBlock(false)
       .setLastPacketInBlock(false)
@@ -76,21 +75,21 @@ public class PacketHeader {
     Preconditions.checkArgument(packetLen >= Ints.BYTES,
     Preconditions.checkArgument(packetLen >= Ints.BYTES,
         "packet len %s should always be at least 4 bytes",
         "packet len %s should always be at least 4 bytes",
         packetLen);
         packetLen);
-    
+
     PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
     PacketHeaderProto.Builder builder = PacketHeaderProto.newBuilder()
-      .setOffsetInBlock(offsetInBlock)
-      .setSeqno(seqno)
-      .setLastPacketInBlock(lastPacketInBlock)
-      .setDataLen(dataLen);
-      
+        .setOffsetInBlock(offsetInBlock)
+        .setSeqno(seqno)
+        .setLastPacketInBlock(lastPacketInBlock)
+        .setDataLen(dataLen);
+
     if (syncBlock) {
     if (syncBlock) {
       // Only set syncBlock if it is specified.
       // Only set syncBlock if it is specified.
       // This is wire-incompatible with Hadoop 2.0.0-alpha due to HDFS-3721
       // 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
       // because it changes the length of the packet header, and BlockReceiver
       // in that version did not support variable-length headers.
       // in that version did not support variable-length headers.
-      builder.setSyncBlock(syncBlock);
+      builder.setSyncBlock(true);
     }
     }
-      
+
     proto = builder.build();
     proto = builder.build();
   }
   }
 
 
@@ -121,16 +120,16 @@ public class PacketHeader {
   @Override
   @Override
   public String toString() {
   public String toString() {
     return "PacketHeader with packetLen=" + packetLen +
     return "PacketHeader with packetLen=" + packetLen +
-      " header data: " + 
+      " header data: " +
       proto.toString();
       proto.toString();
   }
   }
-  
+
   public void setFieldsFromData(
   public void setFieldsFromData(
       int packetLen, byte[] headerData) throws InvalidProtocolBufferException {
       int packetLen, byte[] headerData) throws InvalidProtocolBufferException {
     this.packetLen = packetLen;
     this.packetLen = packetLen;
     proto = PacketHeaderProto.parseFrom(headerData);
     proto = PacketHeaderProto.parseFrom(headerData);
   }
   }
-  
+
   public void readFields(ByteBuffer buf) throws IOException {
   public void readFields(ByteBuffer buf) throws IOException {
     packetLen = buf.getInt();
     packetLen = buf.getInt();
     short protoLen = buf.getShort();
     short protoLen = buf.getShort();
@@ -138,7 +137,7 @@ public class PacketHeader {
     buf.get(data);
     buf.get(data);
     proto = PacketHeaderProto.parseFrom(data);
     proto = PacketHeaderProto.parseFrom(data);
   }
   }
-  
+
   public void readFields(DataInputStream in) throws IOException {
   public void readFields(DataInputStream in) throws IOException {
     this.packetLen = in.readInt();
     this.packetLen = in.readInt();
     short protoLen = in.readShort();
     short protoLen = in.readShort();
@@ -170,7 +169,7 @@ public class PacketHeader {
       throw new RuntimeException(e);
       throw new RuntimeException(e);
     }
     }
   }
   }
-  
+
   public void write(DataOutputStream out) throws IOException {
   public void write(DataOutputStream out) throws IOException {
     assert proto.getSerializedSize() <= MAX_PROTO_SIZE
     assert proto.getSerializedSize() <= MAX_PROTO_SIZE
     : "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
     : "Expected " + (MAX_PROTO_SIZE) + " got: " + proto.getSerializedSize();
@@ -178,7 +177,7 @@ public class PacketHeader {
     out.writeShort(proto.getSerializedSize());
     out.writeShort(proto.getSerializedSize());
     proto.writeTo(out);
     proto.writeTo(out);
   }
   }
-  
+
   public byte[] getBytes() {
   public byte[] getBytes() {
     ByteBuffer buf = ByteBuffer.allocate(getSerializedSize());
     ByteBuffer buf = ByteBuffer.allocate(getSerializedSize());
     putInBuffer(buf);
     putInBuffer(buf);
@@ -187,8 +186,8 @@ public class PacketHeader {
 
 
   /**
   /**
    * Perform a sanity check on the packet, returning true if it is sane.
    * 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) {
   public boolean sanityCheck(long lastSeqNo) {
     // We should only have a non-positive data length for the last packet
     // 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
     // The last packet should not contain data
     if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
     if (proto.getLastPacketInBlock() && proto.getDataLen() != 0) return false;
     // Seqnos should always increase by 1 with each packet received
     // 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
   @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;
   private static final int MAX_PACKET_SIZE = 16 * 1024 * 1024;
 
 
   static final Logger LOG = LoggerFactory.getLogger(PacketReceiver.class);
   static final Logger LOG = LoggerFactory.getLogger(PacketReceiver.class);
-  
+
   private static final DirectBufferPool bufferPool = new DirectBufferPool();
   private static final DirectBufferPool bufferPool = new DirectBufferPool();
   private final boolean useDirectBuffers;
   private final boolean useDirectBuffers;
 
 
@@ -58,12 +58,12 @@ public class PacketReceiver implements Closeable {
    * length prefixes.
    * length prefixes.
    */
    */
   private ByteBuffer curPacketBuf = null;
   private ByteBuffer curPacketBuf = null;
-  
+
   /**
   /**
    * A slice of {@link #curPacketBuf} which contains just the checksums.
    * A slice of {@link #curPacketBuf} which contains just the checksums.
    */
    */
   private ByteBuffer curChecksumSlice = null;
   private ByteBuffer curChecksumSlice = null;
-  
+
   /**
   /**
    * A slice of {@link #curPacketBuf} which contains just the data.
    * 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.
    * The packet header of the most recently read packet.
    */
    */
   private PacketHeader curHeader;
   private PacketHeader curHeader;
-  
+
   public PacketReceiver(boolean useDirectBuffers) {
   public PacketReceiver(boolean useDirectBuffers) {
     this.useDirectBuffers = useDirectBuffers;
     this.useDirectBuffers = useDirectBuffers;
     reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN);
     reallocPacketBuf(PacketHeader.PKT_LENGTHS_LEN);
@@ -86,14 +86,14 @@ public class PacketReceiver implements Closeable {
   public ByteBuffer getDataSlice() {
   public ByteBuffer getDataSlice() {
     return curDataSlice;
     return curDataSlice;
   }
   }
-  
+
   public ByteBuffer getChecksumSlice() {
   public ByteBuffer getChecksumSlice() {
     return curChecksumSlice;
     return curChecksumSlice;
   }
   }
 
 
   /**
   /**
    * Reads all of the data for the next packet into the appropriate buffers.
    * 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
    * 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
    * user data and corresponding checksums. The header will be parsed and
    * set.
    * set.
@@ -134,7 +134,7 @@ public class PacketReceiver implements Closeable {
     doReadFully(ch, in, curPacketBuf);
     doReadFully(ch, in, curPacketBuf);
     curPacketBuf.flip();
     curPacketBuf.flip();
     int payloadLen = curPacketBuf.getInt();
     int payloadLen = curPacketBuf.getInt();
-    
+
     if (payloadLen < Ints.BYTES) {
     if (payloadLen < Ints.BYTES) {
       // The "payload length" includes its own length. Therefore it
       // The "payload length" includes its own length. Therefore it
       // should never be less than 4 bytes
       // should never be less than 4 bytes
@@ -146,7 +146,7 @@ public class PacketReceiver implements Closeable {
     if (headerLen < 0) {
     if (headerLen < 0) {
       throw new IOException("Invalid header length " + headerLen);
       throw new IOException("Invalid header length " + headerLen);
     }
     }
-    
+
     LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
     LOG.trace("readNextPacket: dataPlusChecksumLen={}, headerLen={}",
         dataPlusChecksumLen, headerLen);
         dataPlusChecksumLen, headerLen);
 
 
@@ -177,18 +177,18 @@ public class PacketReceiver implements Closeable {
       curHeader = new PacketHeader();
       curHeader = new PacketHeader();
     }
     }
     curHeader.setFieldsFromData(payloadLen, headerBuf);
     curHeader.setFieldsFromData(payloadLen, headerBuf);
-    
+
     // Compute the sub-slices of the packet
     // Compute the sub-slices of the packet
     int checksumLen = dataPlusChecksumLen - curHeader.getDataLen();
     int checksumLen = dataPlusChecksumLen - curHeader.getDataLen();
     if (checksumLen < 0) {
     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=" +
           "exceeds data length received. dataPlusChecksumLen=" +
-          dataPlusChecksumLen + " header: " + curHeader); 
+          dataPlusChecksumLen + " header: " + curHeader);
     }
     }
-    
+
     reslicePacket(headerLen, checksumLen, curHeader.getDataLen());
     reslicePacket(headerLen, checksumLen, curHeader.getDataLen());
   }
   }
-  
+
   /**
   /**
    * Rewrite the last-read packet on the wire to the given output stream.
    * Rewrite the last-read packet on the wire to the given output stream.
    */
    */
@@ -200,7 +200,7 @@ public class PacketReceiver implements Closeable {
         curPacketBuf.remaining());
         curPacketBuf.remaining());
   }
   }
 
 
-  
+
   private static void doReadFully(ReadableByteChannel ch, InputStream in,
   private static void doReadFully(ReadableByteChannel ch, InputStream in,
       ByteBuffer buf) throws IOException {
       ByteBuffer buf) throws IOException {
     if (ch != null) {
     if (ch != null) {
@@ -222,7 +222,7 @@ public class PacketReceiver implements Closeable {
     //   32-bit  16-bit   <protobuf>  <variable length>
     //   32-bit  16-bit   <protobuf>  <variable length>
     //   |--- lenThroughHeader ----|
     //   |--- lenThroughHeader ----|
     //   |----------- lenThroughChecksums   ----|
     //   |----------- lenThroughChecksums   ----|
-    //   |------------------- lenThroughData    ------| 
+    //   |------------------- lenThroughData    ------|
     int lenThroughHeader = PacketHeader.PKT_LENGTHS_LEN + headerLen;
     int lenThroughHeader = PacketHeader.PKT_LENGTHS_LEN + headerLen;
     int lenThroughChecksums = lenThroughHeader + checksumsLen;
     int lenThroughChecksums = lenThroughHeader + checksumsLen;
     int lenThroughData = lenThroughChecksums + dataLen;
     int lenThroughData = lenThroughChecksums + dataLen;
@@ -242,14 +242,14 @@ public class PacketReceiver implements Closeable {
     curPacketBuf.position(lenThroughChecksums);
     curPacketBuf.position(lenThroughChecksums);
     curPacketBuf.limit(lenThroughData);
     curPacketBuf.limit(lenThroughData);
     curDataSlice = curPacketBuf.slice();
     curDataSlice = curPacketBuf.slice();
-    
+
     // Reset buffer to point to the entirety of the packet (including
     // Reset buffer to point to the entirety of the packet (including
     // length prefixes)
     // length prefixes)
     curPacketBuf.position(0);
     curPacketBuf.position(0);
     curPacketBuf.limit(lenThroughData);
     curPacketBuf.limit(lenThroughData);
   }
   }
 
 
-  
+
   private static void readChannelFully(ReadableByteChannel ch, ByteBuffer buf)
   private static void readChannelFully(ReadableByteChannel ch, ByteBuffer buf)
       throws IOException {
       throws IOException {
     while (buf.remaining() > 0) {
     while (buf.remaining() > 0) {
@@ -259,7 +259,7 @@ public class PacketReceiver implements Closeable {
       }
       }
     }
     }
   }
   }
-  
+
   private void reallocPacketBuf(int atLeastCapacity) {
   private void reallocPacketBuf(int atLeastCapacity) {
     // Realloc the buffer if this packet is longer than the previous
     // Realloc the buffer if this packet is longer than the previous
     // one.
     // one.
@@ -277,12 +277,12 @@ public class PacketReceiver implements Closeable {
         curPacketBuf.flip();
         curPacketBuf.flip();
         newBuf.put(curPacketBuf);
         newBuf.put(curPacketBuf);
       }
       }
-      
+
       returnPacketBufToPool();
       returnPacketBufToPool();
       curPacketBuf = newBuf;
       curPacketBuf = newBuf;
     }
     }
   }
   }
-  
+
   private void returnPacketBufToPool() {
   private void returnPacketBufToPool() {
     if (curPacketBuf != null && curPacketBuf.isDirect()) {
     if (curPacketBuf != null && curPacketBuf.isDirect()) {
       bufferPool.returnBuffer(curPacketBuf);
       bufferPool.returnBuffer(curPacketBuf);
@@ -294,7 +294,7 @@ public class PacketReceiver implements Closeable {
   public void close() {
   public void close() {
     returnPacketBufToPool();
     returnPacketBufToPool();
   }
   }
-  
+
   @Override
   @Override
   protected void finalize() throws Throwable {
   protected void finalize() throws Throwable {
     try {
     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 **/
   /** default constructor **/
   public PipelineAck() {
   public PipelineAck() {
   }
   }
-  
+
   /**
   /**
    * Constructor assuming no next DN in pipeline
    * Constructor assuming no next DN in pipeline
    * @param seqno sequence number
    * @param seqno sequence number
@@ -125,7 +125,7 @@ public class PipelineAck {
       .setDownstreamAckTimeNanos(downstreamAckTimeNanos)
       .setDownstreamAckTimeNanos(downstreamAckTimeNanos)
       .build();
       .build();
   }
   }
-  
+
   /**
   /**
    * Get the sequence number
    * Get the sequence number
    * @return the sequence number
    * @return the sequence number
@@ -133,7 +133,7 @@ public class PipelineAck {
   public long getSeqno() {
   public long getSeqno() {
     return proto.getSeqno();
     return proto.getSeqno();
   }
   }
-  
+
   /**
   /**
    * Get the number of replies
    * Get the number of replies
    * @return the number of replies
    * @return the number of replies
@@ -141,7 +141,7 @@ public class PipelineAck {
   public short getNumOfReplies() {
   public short getNumOfReplies() {
     return (short)proto.getReplyCount();
     return (short)proto.getReplyCount();
   }
   }
-  
+
   /**
   /**
    * get the header flag of ith reply
    * 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.
    * @return null if it is not an OOB ack.
    */
    */
   public Status getOOBStatus() {
   public Status getOOBStatus() {
@@ -216,7 +216,7 @@ public class PipelineAck {
   public void write(OutputStream out) throws IOException {
   public void write(OutputStream out) throws IOException {
     proto.writeDelimitedTo(out);
     proto.writeDelimitedTo(out);
   }
   }
-  
+
   @Override //Object
   @Override //Object
   public String toString() {
   public String toString() {
     return TextFormat.shortDebugString(proto);
     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 final Condition condition;
 
 
-    private Policy(Condition condition) {
+    Policy(Condition condition) {
       this.condition = condition;
       this.condition = condition;
     }
     }
-    
+
     Condition getCondition() {
     Condition getCondition() {
       return condition;
       return condition;
     }
     }
   }
   }
 
 
   /** Datanode replacement condition */
   /** Datanode replacement condition */
-  private static interface Condition {
+  private interface Condition {
     /** Return true unconditionally. */
     /** Return true unconditionally. */
-    static final Condition TRUE = new Condition() {
+    Condition TRUE = new Condition() {
       @Override
       @Override
       public boolean satisfy(short replication, DatanodeInfo[] existings,
       public boolean satisfy(short replication, DatanodeInfo[] existings,
           int nExistings, boolean isAppend, boolean isHflushed) {
           int nExistings, boolean isAppend, boolean isHflushed) {
@@ -64,7 +64,7 @@ public class ReplaceDatanodeOnFailure {
     };
     };
 
 
     /** Return false unconditionally. */
     /** Return false unconditionally. */
-    static final Condition FALSE = new Condition() {
+    Condition FALSE = new Condition() {
       @Override
       @Override
       public boolean satisfy(short replication, DatanodeInfo[] existings,
       public boolean satisfy(short replication, DatanodeInfo[] existings,
           int nExistings, boolean isAppend, boolean isHflushed) {
           int nExistings, boolean isAppend, boolean isHflushed) {
@@ -80,31 +80,24 @@ public class ReplaceDatanodeOnFailure {
      *   (1) floor(r/2) >= n; or
      *   (1) floor(r/2) >= n; or
      *   (2) r > n and the block is hflushed/appended.
      *   (2) r > n and the block is hflushed/appended.
      */
      */
-    static final Condition DEFAULT = new Condition() {
+    Condition DEFAULT = new Condition() {
       @Override
       @Override
       public boolean satisfy(final short replication,
       public boolean satisfy(final short replication,
           final DatanodeInfo[] existings, final int n, final boolean isAppend,
           final DatanodeInfo[] existings, final int n, final boolean isAppend,
           final boolean isHflushed) {
           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? */
     /** 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 Policy policy;
   private final boolean bestEffort;
   private final boolean bestEffort;
-  
+
   public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
   public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
     this.policy = policy;
     this.policy = policy;
     this.bestEffort = bestEffort;
     this.bestEffort = bestEffort;
@@ -124,7 +117,7 @@ public class ReplaceDatanodeOnFailure {
    * Best effort means that the client will try to replace the failed datanode
    * Best effort means that the client will try to replace the failed datanode
    * (provided that the policy is satisfied), however, it will continue the
    * (provided that the policy is satisfied), however, it will continue the
    * write operation in case that the datanode replacement also fails.
    * write operation in case that the datanode replacement also fails.
-   * 
+   *
    * @return Suppose the datanode replacement fails.
    * @return Suppose the datanode replacement fails.
    *     false: An exception should be thrown so that the write will fail.
    *     false: An exception should be thrown so that the write will fail.
    *     true : The write should be resumed with the remaining datandoes.
    *     true : The write should be resumed with the remaining datandoes.
@@ -137,16 +130,13 @@ public class ReplaceDatanodeOnFailure {
   public boolean satisfy(
   public boolean satisfy(
       final short replication, final DatanodeInfo[] existings,
       final short replication, final DatanodeInfo[] existings,
       final boolean isAppend, final boolean isHflushed) {
       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
   @Override
   public String toString() {
   public String toString() {
     return policy.toString();
     return policy.toString();
@@ -158,7 +148,7 @@ public class ReplaceDatanodeOnFailure {
     final boolean bestEffort = conf.getBoolean(
     final boolean bestEffort = conf.getBoolean(
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
-    
+
     return new ReplaceDatanodeOnFailure(policy, bestEffort);
     return new ReplaceDatanodeOnFailure(policy, bestEffort);
   }
   }
 
 
@@ -197,4 +187,4 @@ public class ReplaceDatanodeOnFailure {
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
         HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
         bestEffort);
         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. */
   /** Create a sender for DataTransferProtocol with a output stream. */
   public Sender(final DataOutputStream out) {
   public Sender(final DataOutputStream out) {
-    this.out = out;    
+    this.out = out;
   }
   }
 
 
   /** Initialize a operation. */
   /** 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);
     out.writeShort(DataTransferProtocol.DATA_TRANSFER_VERSION);
     op.write(out);
     op.write(out);
   }
   }
@@ -80,13 +79,14 @@ public class Sender implements DataTransferProtocol {
     out.flush();
     out.flush();
   }
   }
 
 
-  static private CachingStrategyProto getCachingStrategy(CachingStrategy cachingStrategy) {
+  static private CachingStrategyProto getCachingStrategy(
+      CachingStrategy cachingStrategy) {
     CachingStrategyProto.Builder builder = CachingStrategyProto.newBuilder();
     CachingStrategyProto.Builder builder = CachingStrategyProto.newBuilder();
     if (cachingStrategy.getReadahead() != null) {
     if (cachingStrategy.getReadahead() != null) {
-      builder.setReadahead(cachingStrategy.getReadahead().longValue());
+      builder.setReadahead(cachingStrategy.getReadahead());
     }
     }
     if (cachingStrategy.getDropBehind() != null) {
     if (cachingStrategy.getDropBehind() != null) {
-      builder.setDropBehind(cachingStrategy.getDropBehind().booleanValue());
+      builder.setDropBehind(cachingStrategy.getDropBehind());
     }
     }
     return builder.build();
     return builder.build();
   }
   }
@@ -101,24 +101,25 @@ public class Sender implements DataTransferProtocol {
       final CachingStrategy cachingStrategy) throws IOException {
       final CachingStrategy cachingStrategy) throws IOException {
 
 
     OpReadBlockProto proto = OpReadBlockProto.newBuilder()
     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);
     send(out, Op.READ_BLOCK, proto);
   }
   }
-  
+
 
 
   @Override
   @Override
   public void writeBlock(final ExtendedBlock blk,
   public void writeBlock(final ExtendedBlock blk,
-      final StorageType storageType, 
+      final StorageType storageType,
       final Token<BlockTokenIdentifier> blockToken,
       final Token<BlockTokenIdentifier> blockToken,
       final String clientName,
       final String clientName,
       final DatanodeInfo[] targets,
       final DatanodeInfo[] targets,
-      final StorageType[] targetStorageTypes, 
+      final StorageType[] targetStorageTypes,
       final DatanodeInfo source,
       final DatanodeInfo source,
       final BlockConstructionStage stage,
       final BlockConstructionStage stage,
       final int pipelineSize,
       final int pipelineSize,
@@ -132,26 +133,27 @@ public class Sender implements DataTransferProtocol {
       final boolean[] targetPinnings) throws IOException {
       final boolean[] targetPinnings) throws IOException {
     ClientOperationHeaderProto header = DataTransferProtoUtil.buildClientHeader(
     ClientOperationHeaderProto header = DataTransferProtoUtil.buildClientHeader(
         blk, clientName, blockToken);
         blk, clientName, blockToken);
-    
+
     ChecksumProto checksumProto =
     ChecksumProto checksumProto =
-      DataTransferProtoUtil.toProto(requestedChecksum);
+        DataTransferProtoUtil.toProto(requestedChecksum);
 
 
     OpWriteBlockProto.Builder proto = OpWriteBlockProto.newBuilder()
     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) {
     if (source != null) {
       proto.setSource(PBHelperClient.convertDatanodeInfo(source));
       proto.setSource(PBHelperClient.convertDatanodeInfo(source));
     }
     }
@@ -165,13 +167,14 @@ public class Sender implements DataTransferProtocol {
       final String clientName,
       final String clientName,
       final DatanodeInfo[] targets,
       final DatanodeInfo[] targets,
       final StorageType[] targetStorageTypes) throws IOException {
       final StorageType[] targetStorageTypes) throws IOException {
-    
+
     OpTransferBlockProto proto = OpTransferBlockProto.newBuilder()
     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);
     send(out, Op.TRANSFER_BLOCK, proto);
   }
   }
@@ -180,11 +183,11 @@ public class Sender implements DataTransferProtocol {
   public void requestShortCircuitFds(final ExtendedBlock blk,
   public void requestShortCircuitFds(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken,
       final Token<BlockTokenIdentifier> blockToken,
       SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
       SlotId slotId, int maxVersion, boolean supportsReceiptVerification)
-        throws IOException {
+      throws IOException {
     OpRequestShortCircuitAccessProto.Builder builder =
     OpRequestShortCircuitAccessProto.Builder builder =
         OpRequestShortCircuitAccessProto.newBuilder()
         OpRequestShortCircuitAccessProto.newBuilder()
-          .setHeader(DataTransferProtoUtil.buildBaseHeader(
-            blk, blockToken)).setMaxVersion(maxVersion);
+            .setHeader(DataTransferProtoUtil.buildBaseHeader(
+                blk, blockToken)).setMaxVersion(maxVersion);
     if (slotId != null) {
     if (slotId != null) {
       builder.setSlotId(PBHelperClient.convert(slotId));
       builder.setSlotId(PBHelperClient.convert(slotId));
     }
     }
@@ -192,12 +195,12 @@ public class Sender implements DataTransferProtocol {
     OpRequestShortCircuitAccessProto proto = builder.build();
     OpRequestShortCircuitAccessProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
     send(out, Op.REQUEST_SHORT_CIRCUIT_FDS, proto);
   }
   }
-  
+
   @Override
   @Override
   public void releaseShortCircuitFds(SlotId slotId) throws IOException {
   public void releaseShortCircuitFds(SlotId slotId) throws IOException {
     ReleaseShortCircuitAccessRequestProto.Builder builder =
     ReleaseShortCircuitAccessRequestProto.Builder builder =
         ReleaseShortCircuitAccessRequestProto.newBuilder().
         ReleaseShortCircuitAccessRequestProto.newBuilder().
-        setSlotId(PBHelperClient.convert(slotId));
+            setSlotId(PBHelperClient.convert(slotId));
     SpanId spanId = Tracer.getCurrentSpanId();
     SpanId spanId = Tracer.getCurrentSpanId();
     if (spanId.isValid()) {
     if (spanId.isValid()) {
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
@@ -212,7 +215,7 @@ public class Sender implements DataTransferProtocol {
   public void requestShortCircuitShm(String clientName) throws IOException {
   public void requestShortCircuitShm(String clientName) throws IOException {
     ShortCircuitShmRequestProto.Builder builder =
     ShortCircuitShmRequestProto.Builder builder =
         ShortCircuitShmRequestProto.newBuilder().
         ShortCircuitShmRequestProto.newBuilder().
-        setClientName(clientName);
+            setClientName(clientName);
     SpanId spanId = Tracer.getCurrentSpanId();
     SpanId spanId = Tracer.getCurrentSpanId();
     if (spanId.isValid()) {
     if (spanId.isValid()) {
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
       builder.setTraceInfo(DataTransferTraceInfoProto.newBuilder().
@@ -222,20 +225,20 @@ public class Sender implements DataTransferProtocol {
     ShortCircuitShmRequestProto proto = builder.build();
     ShortCircuitShmRequestProto proto = builder.build();
     send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);
     send(out, Op.REQUEST_SHORT_CIRCUIT_SHM, proto);
   }
   }
-  
+
   @Override
   @Override
   public void replaceBlock(final ExtendedBlock blk,
   public void replaceBlock(final ExtendedBlock blk,
-      final StorageType storageType, 
+      final StorageType storageType,
       final Token<BlockTokenIdentifier> blockToken,
       final Token<BlockTokenIdentifier> blockToken,
       final String delHint,
       final String delHint,
       final DatanodeInfo source) throws IOException {
       final DatanodeInfo source) throws IOException {
     OpReplaceBlockProto proto = OpReplaceBlockProto.newBuilder()
     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);
     send(out, Op.REPLACE_BLOCK, proto);
   }
   }
 
 
@@ -243,9 +246,9 @@ public class Sender implements DataTransferProtocol {
   public void copyBlock(final ExtendedBlock blk,
   public void copyBlock(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken) throws IOException {
       final Token<BlockTokenIdentifier> blockToken) throws IOException {
     OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
     OpCopyBlockProto proto = OpCopyBlockProto.newBuilder()
-      .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
-      .build();
-    
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .build();
+
     send(out, Op.COPY_BLOCK, proto);
     send(out, Op.COPY_BLOCK, proto);
   }
   }
 
 
@@ -253,9 +256,9 @@ public class Sender implements DataTransferProtocol {
   public void blockChecksum(final ExtendedBlock blk,
   public void blockChecksum(final ExtendedBlock blk,
       final Token<BlockTokenIdentifier> blockToken) throws IOException {
       final Token<BlockTokenIdentifier> blockToken) throws IOException {
     OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
     OpBlockChecksumProto proto = OpBlockChecksumProto.newBuilder()
-      .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
-      .build();
-    
+        .setHeader(DataTransferProtoUtil.buildBaseHeader(blk, blockToken))
+        .build();
+
     send(out, Op.BLOCK_CHECKSUM, proto);
     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 default implementation is to return false indicating that
  * the channel is not trusted.
  * the channel is not trusted.
  * This class can be overridden to provide custom logic to determine
  * 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.
  * The custom class can be specified via configuration.
  *
  *
  */
  */
@@ -39,14 +39,13 @@ public class TrustedChannelResolver implements Configurable {
   /**
   /**
    * Returns an instance of TrustedChannelResolver.
    * Returns an instance of TrustedChannelResolver.
    * Looks up the configuration to see if there is custom class specified.
    * Looks up the configuration to see if there is custom class specified.
-   * @param conf
    * @return TrustedChannelResolver
    * @return TrustedChannelResolver
    */
    */
   public static TrustedChannelResolver getInstance(Configuration conf) {
   public static TrustedChannelResolver getInstance(Configuration conf) {
     Class<? extends TrustedChannelResolver> clazz =
     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);
     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
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
  * 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
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -69,7 +69,7 @@ import com.google.protobuf.ByteString;
 public final class DataTransferSaslUtil {
 public final class DataTransferSaslUtil {
 
 
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
-    DataTransferSaslUtil.class);
+      DataTransferSaslUtil.class);
 
 
   /**
   /**
    * Delimiter for the three-part SASL username string.
    * Delimiter for the three-part SASL username string.
@@ -97,20 +97,20 @@ public final class DataTransferSaslUtil {
       throw new IOException("Failed to complete SASL handshake");
       throw new IOException("Failed to complete SASL handshake");
     }
     }
     Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
     Set<String> requestedQop = ImmutableSet.copyOf(Arrays.asList(
-      saslProps.get(Sasl.QOP).split(",")));
+        saslProps.get(Sasl.QOP).split(",")));
     String negotiatedQop = sasl.getNegotiatedQop();
     String negotiatedQop = sasl.getNegotiatedQop();
     LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
     LOG.debug("Verifying QOP, requested QOP = {}, negotiated QOP = {}",
-      requestedQop, negotiatedQop);
+        requestedQop, negotiatedQop);
     if (!requestedQop.contains(negotiatedQop)) {
     if (!requestedQop.contains(negotiatedQop)) {
       throw new IOException(String.format("SASL handshake completed, but " +
       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.
    * Check whether requested SASL Qop contains privacy.
-   * 
+   *
    * @param saslProps properties of SASL negotiation
    * @param saslProps properties of SASL negotiation
    * @return boolean true if privacy exists
    * @return boolean true if privacy exists
    */
    */
@@ -145,7 +145,7 @@ public final class DataTransferSaslUtil {
    */
    */
   public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
   public static char[] encryptionKeyToPassword(byte[] encryptionKey) {
     return new String(Base64.encodeBase64(encryptionKey, false), Charsets.UTF_8)
     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
    * [host][/ip-address]:port.  The host may be missing.  The IP address (and
    * preceding '/') may be missing.  The port preceded by ':' is always present.
    * preceding '/') may be missing.  The port preceded by ':' is always present.
    *
    *
-   * @param peer
    * @return InetAddress from peer
    * @return InetAddress from peer
    */
    */
   public static InetAddress getPeerAddress(Peer peer) {
   public static InetAddress getPeerAddress(Peer peer) {
@@ -181,23 +180,26 @@ public final class DataTransferSaslUtil {
     String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
     String qops = conf.get(DFS_DATA_TRANSFER_PROTECTION_KEY);
     if (qops == null || qops.isEmpty()) {
     if (qops == null || qops.isEmpty()) {
       LOG.debug("DataTransferProtocol not using SaslPropertiesResolver, no " +
       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;
       return null;
     }
     }
     Configuration saslPropsResolverConf = new Configuration(conf);
     Configuration saslPropsResolverConf = new Configuration(conf);
     saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
     saslPropsResolverConf.set(HADOOP_RPC_PROTECTION, qops);
     Class<? extends SaslPropertiesResolver> resolverClass = conf.getClass(
     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,
     saslPropsResolverConf.setClass(HADOOP_SECURITY_SASL_PROPS_RESOLVER_CLASS,
-      resolverClass, SaslPropertiesResolver.class);
+        resolverClass, SaslPropertiesResolver.class);
     SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
     SaslPropertiesResolver resolver = SaslPropertiesResolver.getInstance(
-      saslPropsResolverConf);
+        saslPropsResolverConf);
     LOG.debug("DataTransferProtocol using SaslPropertiesResolver, configured " +
     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;
     return resolver;
   }
   }
 
 
@@ -219,10 +221,10 @@ public final class DataTransferSaslUtil {
       return proto.getPayload().toByteArray();
       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 in stream to read
    * @param cipherOptions list to store negotiation cipher options
    * @param cipherOptions list to store negotiation cipher options
    * @return byte[] SASL negotiation message
    * @return byte[] SASL negotiation message
@@ -246,10 +248,10 @@ public final class DataTransferSaslUtil {
       return proto.getPayload().toByteArray();
       return proto.getPayload().toByteArray();
     }
     }
   }
   }
-  
+
   /**
   /**
    * Negotiate a cipher option which server supports.
    * Negotiate a cipher option which server supports.
-   * 
+   *
    * @param conf the configuration
    * @param conf the configuration
    * @param options the cipher options which client supports
    * @param options the cipher options which client supports
    * @return CipherOption negotiated cipher option
    * @return CipherOption negotiated cipher option
@@ -279,6 +281,7 @@ public final class DataTransferSaslUtil {
           byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
           byte[] inIv = new byte[suite.getAlgorithmBlockSize()];
           byte[] outKey = new byte[keyLen];
           byte[] outKey = new byte[keyLen];
           byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
           byte[] outIv = new byte[suite.getAlgorithmBlockSize()];
+          assert codec != null;
           codec.generateSecureRandom(inKey);
           codec.generateSecureRandom(inKey);
           codec.generateSecureRandom(inIv);
           codec.generateSecureRandom(inIv);
           codec.generateSecureRandom(outKey);
           codec.generateSecureRandom(outKey);
@@ -289,21 +292,21 @@ public final class DataTransferSaslUtil {
     }
     }
     return null;
     return null;
   }
   }
-  
+
   /**
   /**
    * Send SASL message and negotiated cipher option to client.
    * Send SASL message and negotiated cipher option to client.
-   * 
+   *
    * @param out stream to receive message
    * @param out stream to receive message
    * @param payload to send
    * @param payload to send
    * @param option negotiated cipher option
    * @param option negotiated cipher option
    * @throws IOException for any error
    * @throws IOException for any error
    */
    */
   public static void sendSaslMessageAndNegotiatedCipherOption(
   public static void sendSaslMessageAndNegotiatedCipherOption(
-      OutputStream out, byte[] payload, CipherOption option) 
-          throws IOException {
+      OutputStream out, byte[] payload, CipherOption option)
+      throws IOException {
     DataTransferEncryptorMessageProto.Builder builder =
     DataTransferEncryptorMessageProto.Builder builder =
         DataTransferEncryptorMessageProto.newBuilder();
         DataTransferEncryptorMessageProto.newBuilder();
-    
+
     builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
     builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
     if (payload != null) {
     if (payload != null) {
       builder.setPayload(ByteString.copyFrom(payload));
       builder.setPayload(ByteString.copyFrom(payload));
@@ -311,16 +314,16 @@ public final class DataTransferSaslUtil {
     if (option != null) {
     if (option != null) {
       builder.addCipherOption(PBHelperClient.convert(option));
       builder.addCipherOption(PBHelperClient.convert(option));
     }
     }
-    
+
     DataTransferEncryptorMessageProto proto = builder.build();
     DataTransferEncryptorMessageProto proto = builder.build();
     proto.writeDelimitedTo(out);
     proto.writeDelimitedTo(out);
     out.flush();
     out.flush();
   }
   }
-  
+
   /**
   /**
    * Create IOStreamPair of {@link org.apache.hadoop.crypto.CryptoInputStream}
    * Create IOStreamPair of {@link org.apache.hadoop.crypto.CryptoInputStream}
    * and {@link org.apache.hadoop.crypto.CryptoOutputStream}
    * and {@link org.apache.hadoop.crypto.CryptoOutputStream}
-   * 
+   *
    * @param conf the configuration
    * @param conf the configuration
    * @param cipherOption negotiated cipher option
    * @param cipherOption negotiated cipher option
    * @param out underlying output stream
    * @param out underlying output stream
@@ -330,7 +333,7 @@ public final class DataTransferSaslUtil {
    * @throws IOException for any error
    * @throws IOException for any error
    */
    */
   public static IOStreamPair createStreamPair(Configuration conf,
   public static IOStreamPair createStreamPair(Configuration conf,
-      CipherOption cipherOption, OutputStream out, InputStream in, 
+      CipherOption cipherOption, OutputStream out, InputStream in,
       boolean isServer) throws IOException {
       boolean isServer) throws IOException {
     LOG.debug("Creating IOStreamPair of CryptoInputStream and "
     LOG.debug("Creating IOStreamPair of CryptoInputStream and "
         + "CryptoOutputStream.");
         + "CryptoOutputStream.");
@@ -340,9 +343,9 @@ public final class DataTransferSaslUtil {
     byte[] inIv = cipherOption.getInIv();
     byte[] inIv = cipherOption.getInIv();
     byte[] outKey = cipherOption.getOutKey();
     byte[] outKey = cipherOption.getOutKey();
     byte[] outIv = cipherOption.getOutIv();
     byte[] outIv = cipherOption.getOutIv();
-    InputStream cIn = new CryptoInputStream(in, codec, 
+    InputStream cIn = new CryptoInputStream(in, codec,
         isServer ? inKey : outKey, isServer ? inIv : outIv);
         isServer ? inKey : outKey, isServer ? inIv : outIv);
-    OutputStream cOut = new CryptoOutputStream(out, codec, 
+    OutputStream cOut = new CryptoOutputStream(out, codec,
         isServer ? outKey : inKey, isServer ? outIv : inIv);
         isServer ? outKey : inKey, isServer ? outIv : inIv);
     return new IOStreamPair(cIn, cOut);
     return new IOStreamPair(cIn, cOut);
   }
   }
@@ -370,10 +373,10 @@ public final class DataTransferSaslUtil {
       throws IOException {
       throws IOException {
     sendSaslMessage(out, DataTransferEncryptorStatus.SUCCESS, payload, null);
     sendSaslMessage(out, DataTransferEncryptorStatus.SUCCESS, payload, null);
   }
   }
-  
+
   /**
   /**
    * Send a SASL negotiation message and negotiation cipher options to server.
    * Send a SASL negotiation message and negotiation cipher options to server.
-   * 
+   *
    * @param out stream to receive message
    * @param out stream to receive message
    * @param payload to send
    * @param payload to send
    * @param options cipher options to negotiate
    * @param options cipher options to negotiate
@@ -381,10 +384,10 @@ public final class DataTransferSaslUtil {
    */
    */
   public static void sendSaslMessageAndNegotiationCipherOptions(
   public static void sendSaslMessageAndNegotiationCipherOptions(
       OutputStream out, byte[] payload, List<CipherOption> options)
       OutputStream out, byte[] payload, List<CipherOption> options)
-          throws IOException {
+      throws IOException {
     DataTransferEncryptorMessageProto.Builder builder =
     DataTransferEncryptorMessageProto.Builder builder =
         DataTransferEncryptorMessageProto.newBuilder();
         DataTransferEncryptorMessageProto.newBuilder();
-    
+
     builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
     builder.setStatus(DataTransferEncryptorStatus.SUCCESS);
     if (payload != null) {
     if (payload != null) {
       builder.setPayload(ByteString.copyFrom(payload));
       builder.setPayload(ByteString.copyFrom(payload));
@@ -392,23 +395,23 @@ public final class DataTransferSaslUtil {
     if (options != null) {
     if (options != null) {
       builder.addAllCipherOption(PBHelperClient.convertCipherOptions(options));
       builder.addAllCipherOption(PBHelperClient.convertCipherOptions(options));
     }
     }
-    
+
     DataTransferEncryptorMessageProto proto = builder.build();
     DataTransferEncryptorMessageProto proto = builder.build();
     proto.writeDelimitedTo(out);
     proto.writeDelimitedTo(out);
     out.flush();
     out.flush();
   }
   }
-  
+
   /**
   /**
    * Read SASL message and negotiated cipher option from server.
    * Read SASL message and negotiated cipher option from server.
-   * 
+   *
    * @param in stream to read
    * @param in stream to read
-   * @return SaslResponseWithNegotiatedCipherOption SASL message and 
+   * @return SaslResponseWithNegotiatedCipherOption SASL message and
    * negotiated cipher option
    * negotiated cipher option
    * @throws IOException for any error
    * @throws IOException for any error
    */
    */
   public static SaslResponseWithNegotiatedCipherOption
   public static SaslResponseWithNegotiatedCipherOption
       readSaslMessageAndNegotiatedCipherOption(InputStream in)
       readSaslMessageAndNegotiatedCipherOption(InputStream in)
-          throws IOException {
+      throws IOException {
     DataTransferEncryptorMessageProto proto =
     DataTransferEncryptorMessageProto proto =
         DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
         DataTransferEncryptorMessageProto.parseFrom(vintPrefixed(in));
     if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
     if (proto.getStatus() == DataTransferEncryptorStatus.ERROR_UNKNOWN_KEY) {
@@ -426,17 +429,17 @@ public final class DataTransferSaslUtil {
       return new SaslResponseWithNegotiatedCipherOption(response, option);
       return new SaslResponseWithNegotiatedCipherOption(response, option);
     }
     }
   }
   }
-  
+
   /**
   /**
    * Encrypt the key and iv of the negotiated cipher option.
    * Encrypt the key and iv of the negotiated cipher option.
-   * 
+   *
    * @param option negotiated cipher option
    * @param option negotiated cipher option
    * @param sasl SASL participant representing server
    * @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
    * encrypted key and iv
    * @throws IOException for any error
    * @throws IOException for any error
    */
    */
-  public static CipherOption wrap(CipherOption option, SaslParticipant sasl) 
+  public static CipherOption wrap(CipherOption option, SaslParticipant sasl)
       throws IOException {
       throws IOException {
     if (option != null) {
     if (option != null) {
       byte[] inKey = option.getInKey();
       byte[] inKey = option.getInKey();
@@ -450,16 +453,16 @@ public final class DataTransferSaslUtil {
       return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
       return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
           outKey, option.getOutIv());
           outKey, option.getOutIv());
     }
     }
-    
+
     return null;
     return null;
   }
   }
-  
+
   /**
   /**
    * Decrypt the key and iv of the negotiated cipher option.
    * Decrypt the key and iv of the negotiated cipher option.
-   * 
+   *
    * @param option negotiated cipher option
    * @param option negotiated cipher option
    * @param sasl SASL participant representing client
    * @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
    * decrypted key and iv
    * @throws IOException for any error
    * @throws IOException for any error
    */
    */
@@ -477,7 +480,7 @@ public final class DataTransferSaslUtil {
       return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
       return new CipherOption(option.getCipherSuite(), inKey, option.getInIv(),
           outKey, option.getOutIv());
           outKey, option.getOutIv());
     }
     }
-    
+
     return null;
     return null;
   }
   }
 
 
@@ -492,10 +495,10 @@ public final class DataTransferSaslUtil {
    */
    */
   public static void sendSaslMessage(OutputStream out,
   public static void sendSaslMessage(OutputStream out,
       DataTransferEncryptorStatus status, byte[] payload, String message)
       DataTransferEncryptorStatus status, byte[] payload, String message)
-          throws IOException {
+      throws IOException {
     DataTransferEncryptorMessageProto.Builder builder =
     DataTransferEncryptorMessageProto.Builder builder =
         DataTransferEncryptorMessageProto.newBuilder();
         DataTransferEncryptorMessageProto.newBuilder();
-    
+
     builder.setStatus(status);
     builder.setStatus(status);
     if (payload != null) {
     if (payload != null) {
       builder.setPayload(ByteString.copyFrom(payload));
       builder.setPayload(ByteString.copyFrom(payload));
@@ -503,7 +506,7 @@ public final class DataTransferSaslUtil {
     if (message != null) {
     if (message != null) {
       builder.setMessage(message);
       builder.setMessage(message);
     }
     }
-    
+
     DataTransferEncryptorMessageProto proto = builder.build();
     DataTransferEncryptorMessageProto proto = builder.build();
     proto.writeDelimitedTo(out);
     proto.writeDelimitedTo(out);
     out.flush();
     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 {
 public class SaslDataTransferClient {
 
 
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
-    SaslDataTransferClient.class);
+      SaslDataTransferClient.class);
 
 
   private final Configuration conf;
   private final Configuration conf;
   private final AtomicBoolean fallbackToSimpleAuth;
   private final AtomicBoolean fallbackToSimpleAuth;
@@ -94,7 +94,7 @@ public class SaslDataTransferClient {
    * @param trustedChannelResolver for identifying trusted connections that do
    * @param trustedChannelResolver for identifying trusted connections that do
    *   not require SASL negotiation
    *   not require SASL negotiation
    */
    */
-  public SaslDataTransferClient(Configuration conf, 
+  public SaslDataTransferClient(Configuration conf,
       SaslPropertiesResolver saslPropsResolver,
       SaslPropertiesResolver saslPropsResolver,
       TrustedChannelResolver trustedChannelResolver) {
       TrustedChannelResolver trustedChannelResolver) {
     this(conf, saslPropsResolver, trustedChannelResolver, null);
     this(conf, saslPropsResolver, trustedChannelResolver, null);
@@ -110,7 +110,7 @@ public class SaslDataTransferClient {
    * @param fallbackToSimpleAuth checked on each attempt at general SASL
    * @param fallbackToSimpleAuth checked on each attempt at general SASL
    *   handshake, if true forces use of simple auth
    *   handshake, if true forces use of simple auth
    */
    */
-  public SaslDataTransferClient(Configuration conf, 
+  public SaslDataTransferClient(Configuration conf,
       SaslPropertiesResolver saslPropsResolver,
       SaslPropertiesResolver saslPropsResolver,
       TrustedChannelResolver trustedChannelResolver,
       TrustedChannelResolver trustedChannelResolver,
       AtomicBoolean fallbackToSimpleAuth) {
       AtomicBoolean fallbackToSimpleAuth) {
@@ -138,9 +138,9 @@ public class SaslDataTransferClient {
       throws IOException {
       throws IOException {
     // The encryption key factory only returns a key if encryption is enabled.
     // The encryption key factory only returns a key if encryption is enabled.
     DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
     DataEncryptionKey encryptionKey = !trustedChannelResolver.isTrusted() ?
-      encryptionKeyFactory.newDataEncryptionKey() : null;
+        encryptionKeyFactory.newDataEncryptionKey() : null;
     IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
     IOStreamPair ios = send(socket.getInetAddress(), underlyingOut,
-      underlyingIn, encryptionKey, accessToken, datanodeId);
+        underlyingIn, encryptionKey, accessToken, datanodeId);
     return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
     return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
   }
   }
 
 
@@ -158,8 +158,8 @@ public class SaslDataTransferClient {
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       throws IOException {
       throws IOException {
     IOStreamPair ios = checkTrustAndSend(getPeerAddress(peer),
     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.
     // TODO: Consider renaming EncryptedPeer to SaslPeer.
     return ios != null ? new EncryptedPeer(peer, ios) : peer;
     return ios != null ? new EncryptedPeer(peer, ios) : peer;
   }
   }
@@ -181,7 +181,7 @@ public class SaslDataTransferClient {
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       throws IOException {
       throws IOException {
     IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
     IOStreamPair ios = checkTrustAndSend(socket.getInetAddress(), underlyingOut,
-      underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
+        underlyingIn, encryptionKeyFactory, accessToken, datanodeId);
     return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
     return ios != null ? ios : new IOStreamPair(underlyingIn, underlyingOut);
   }
   }
 
 
@@ -207,13 +207,13 @@ public class SaslDataTransferClient {
         !trustedChannelResolver.isTrusted(addr)) {
         !trustedChannelResolver.isTrusted(addr)) {
       // The encryption key factory only returns a key if encryption is enabled.
       // The encryption key factory only returns a key if encryption is enabled.
       DataEncryptionKey encryptionKey =
       DataEncryptionKey encryptionKey =
-        encryptionKeyFactory.newDataEncryptionKey();
+          encryptionKeyFactory.newDataEncryptionKey();
       return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
       return send(addr, underlyingOut, underlyingIn, encryptionKey, accessToken,
-        datanodeId);
+          datanodeId);
     } else {
     } else {
       LOG.debug(
       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;
       return null;
     }
     }
   }
   }
@@ -236,40 +236,38 @@ public class SaslDataTransferClient {
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
       throws IOException {
       throws IOException {
     if (encryptionKey != null) {
     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,
       return getEncryptedStreams(underlyingOut, underlyingIn,
-        encryptionKey);
+          encryptionKey);
     } else if (!UserGroupInformation.isSecurityEnabled()) {
     } 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;
       return null;
     } else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
     } else if (SecurityUtil.isPrivilegedPort(datanodeId.getXferPort())) {
       LOG.debug(
       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;
       return null;
     } else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
     } else if (fallbackToSimpleAuth != null && fallbackToSimpleAuth.get()) {
       LOG.debug(
       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;
       return null;
     } else if (saslPropsResolver != null) {
     } else if (saslPropsResolver != null) {
       LOG.debug(
       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 {
     } else {
       // It's a secured cluster using non-privileged ports, but no SASL.  The
       // It's a secured cluster using non-privileged ports, but no SASL.  The
       // only way this can happen is if the DataNode has
       // 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;
       return null;
     }
     }
   }
   }
@@ -287,24 +285,24 @@ public class SaslDataTransferClient {
       InputStream underlyingIn, DataEncryptionKey encryptionKey)
       InputStream underlyingIn, DataEncryptionKey encryptionKey)
       throws IOException {
       throws IOException {
     Map<String, String> saslProps = createSaslPropertiesForEncryption(
     Map<String, String> saslProps = createSaslPropertiesForEncryption(
-      encryptionKey.encryptionAlgorithm);
+        encryptionKey.encryptionAlgorithm);
 
 
     LOG.debug("Client using encryption algorithm {}",
     LOG.debug("Client using encryption algorithm {}",
-      encryptionKey.encryptionAlgorithm);
+        encryptionKey.encryptionAlgorithm);
 
 
     String userName = getUserNameFromEncryptionKey(encryptionKey);
     String userName = getUserNameFromEncryptionKey(encryptionKey);
     char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
     char[] password = encryptionKeyToPassword(encryptionKey.encryptionKey);
     CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
     CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
-      password);
+        password);
     return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
     return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
-      callbackHandler);
+        callbackHandler);
   }
   }
 
 
   /**
   /**
    * The SASL username for an encrypted handshake consists of the keyId,
    * The SASL username for an encrypted handshake consists of the keyId,
    * blockPoolId, and nonce with the first two encoded as Strings, and the third
    * 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.
    * encoded using Base64. The fields are each separated by a single space.
-   * 
+   *
    * @param encryptionKey the encryption key to encode as a SASL username.
    * @param encryptionKey the encryption key to encode as a SASL username.
    * @return encoded username containing keyId, blockPoolId, and nonce
    * @return encoded username containing keyId, blockPoolId, and nonce
    */
    */
@@ -312,7 +310,8 @@ public class SaslDataTransferClient {
       DataEncryptionKey encryptionKey) {
       DataEncryptionKey encryptionKey) {
     return encryptionKey.keyId + NAME_DELIMITER +
     return encryptionKey.keyId + NAME_DELIMITER +
         encryptionKey.blockPoolId + 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.
      * Creates a new SaslClientCallbackHandler.
      *
      *
      * @param userName SASL user name
      * @param userName SASL user name
-     * @Param password SASL password
+     * @param password SASL password
      */
      */
     public SaslClientCallbackHandler(String userName, char[] password) {
     public SaslClientCallbackHandler(String userName, char[] password) {
       this.password = password;
       this.password = password;
@@ -342,15 +341,13 @@ public class SaslDataTransferClient {
       PasswordCallback pc = null;
       PasswordCallback pc = null;
       RealmCallback rc = null;
       RealmCallback rc = null;
       for (Callback callback : callbacks) {
       for (Callback callback : callbacks) {
-        if (callback instanceof RealmChoiceCallback) {
-          continue;
-        } else if (callback instanceof NameCallback) {
+        if (callback instanceof NameCallback) {
           nc = (NameCallback) callback;
           nc = (NameCallback) callback;
         } else if (callback instanceof PasswordCallback) {
         } else if (callback instanceof PasswordCallback) {
           pc = (PasswordCallback) callback;
           pc = (PasswordCallback) callback;
         } else if (callback instanceof RealmCallback) {
         } else if (callback instanceof RealmCallback) {
           rc = (RealmCallback) callback;
           rc = (RealmCallback) callback;
-        } else {
+        } else if (!(callback instanceof RealmChoiceCallback)) {
           throw new UnsupportedCallbackException(callback,
           throw new UnsupportedCallbackException(callback,
               "Unrecognized SASL client callback");
               "Unrecognized SASL client callback");
         }
         }
@@ -374,22 +371,21 @@ public class SaslDataTransferClient {
    * @param underlyingOut connection output stream
    * @param underlyingOut connection output stream
    * @param underlyingIn connection input stream
    * @param underlyingIn connection input stream
    * @param accessToken connection block access token
    * @param accessToken connection block access token
-   * @param datanodeId ID of destination DataNode
    * @return new pair of streams, wrapped after SASL negotiation
    * @return new pair of streams, wrapped after SASL negotiation
    * @throws IOException for any error
    * @throws IOException for any error
    */
    */
   private IOStreamPair getSaslStreams(InetAddress addr,
   private IOStreamPair getSaslStreams(InetAddress addr,
       OutputStream underlyingOut, InputStream underlyingIn,
       OutputStream underlyingOut, InputStream underlyingIn,
-      Token<BlockTokenIdentifier> accessToken, DatanodeID datanodeId)
+      Token<BlockTokenIdentifier> accessToken)
       throws IOException {
       throws IOException {
     Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
     Map<String, String> saslProps = saslPropsResolver.getClientProperties(addr);
 
 
     String userName = buildUserName(accessToken);
     String userName = buildUserName(accessToken);
     char[] password = buildClientPassword(accessToken);
     char[] password = buildClientPassword(accessToken);
     CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
     CallbackHandler callbackHandler = new SaslClientCallbackHandler(userName,
-      password);
+        password);
     return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
     return doSaslHandshake(underlyingOut, underlyingIn, userName, saslProps,
-      callbackHandler);
+        callbackHandler);
   }
   }
 
 
   /**
   /**
@@ -404,7 +400,7 @@ public class SaslDataTransferClient {
    */
    */
   private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
   private static String buildUserName(Token<BlockTokenIdentifier> blockToken) {
     return new String(Base64.encodeBase64(blockToken.getIdentifier(), false),
     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
    * @param blockToken for block access
    * @return SASL password
    * @return SASL password
-   */    
+   */
   private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
   private char[] buildClientPassword(Token<BlockTokenIdentifier> blockToken) {
     return new String(Base64.encodeBase64(blockToken.getPassword(), false),
     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);
     DataInputStream in = new DataInputStream(underlyingIn);
 
 
     SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
     SaslParticipant sasl= SaslParticipant.createClientSaslParticipant(userName,
-      saslProps, callbackHandler);
+        saslProps, callbackHandler);
 
 
     out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
     out.writeInt(SASL_TRANSFER_MAGIC_NUMBER);
     out.flush();
     out.flush();
@@ -467,11 +463,11 @@ public class SaslDataTransferClient {
           cipherOptions.add(option);
           cipherOptions.add(option);
         }
         }
       }
       }
-      sendSaslMessageAndNegotiationCipherOptions(out, localResponse, 
+      sendSaslMessageAndNegotiationCipherOptions(out, localResponse,
           cipherOptions);
           cipherOptions);
 
 
       // step 2 (client-side only)
       // step 2 (client-side only)
-      SaslResponseWithNegotiatedCipherOption response = 
+      SaslResponseWithNegotiatedCipherOption response =
           readSaslMessageAndNegotiatedCipherOption(in);
           readSaslMessageAndNegotiatedCipherOption(in);
       localResponse = sasl.evaluateChallengeOrResponse(response.payload);
       localResponse = sasl.evaluateChallengeOrResponse(response.payload);
       assert localResponse == null;
       assert localResponse == null;
@@ -485,11 +481,11 @@ public class SaslDataTransferClient {
         cipherOption = unwrap(response.cipherOption, sasl);
         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.
       // stream pair.
       return cipherOption != null ? createStreamPair(
       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) {
     } catch (IOException ioe) {
       sendGenericSaslErrorMessage(out, ioe.getMessage());
       sendGenericSaslErrorMessage(out, ioe.getMessage());
       throw ioe;
       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);
       return (String) saslServer.getNegotiatedProperty(Sasl.QOP);
     }
     }
   }
   }
-  
+
   /**
   /**
    * After successful SASL negotiation, returns whether it's QOP privacy
    * After successful SASL negotiation, returns whether it's QOP privacy
-   * 
+   *
    * @return boolean whether it's QOP privacy
    * @return boolean whether it's QOP privacy
    */
    */
   public boolean isNegotiatedQopPrivacy() {
   public boolean isNegotiatedQopPrivacy() {
     String qop = getNegotiatedQop();
     String qop = getNegotiatedQop();
     return qop != null && "auth-conf".equalsIgnoreCase(qop);
     return qop != null && "auth-conf".equalsIgnoreCase(qop);
   }
   }
-  
+
   /**
   /**
    * Wraps a byte array.
    * Wraps a byte array.
-   * 
+   *
    * @param bytes The array containing the bytes to wrap.
    * @param bytes The array containing the bytes to wrap.
    * @param off The starting position at the array
    * @param off The starting position at the array
    * @param len The number of bytes to wrap
    * @param len The number of bytes to wrap
@@ -156,10 +156,10 @@ class SaslParticipant {
       return saslServer.wrap(bytes, off, len);
       return saslServer.wrap(bytes, off, len);
     }
     }
   }
   }
-  
+
   /**
   /**
    * Unwraps a byte array.
    * Unwraps a byte array.
-   * 
+   *
    * @param bytes The array containing the bytes to unwrap.
    * @param bytes The array containing the bytes to unwrap.
    * @param off The starting position at the array
    * @param off The starting position at the array
    * @param len The number of bytes to unwrap
    * @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 {
 public class SaslResponseWithNegotiatedCipherOption {
   final byte[] payload;
   final byte[] payload;
   final CipherOption cipherOption;
   final CipherOption cipherOption;
-  
-  public SaslResponseWithNegotiatedCipherOption(byte[] payload, 
+
+  public SaslResponseWithNegotiatedCipherOption(byte[] payload,
       CipherOption cipherOption) {
       CipherOption cipherOption) {
     this.payload = payload;
     this.payload = payload;
     this.cipherOption = cipherOption;
     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(
 @KerberosInfo(
     serverPrincipal = HdfsClientConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY)
     serverPrincipal = HdfsClientConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY)
 @TokenInfo(BlockTokenSelector.class)
 @TokenInfo(BlockTokenSelector.class)
-@ProtocolInfo(protocolName = 
+@ProtocolInfo(protocolName =
     "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol",
     "org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol",
     protocolVersion = 1)
     protocolVersion = 1)
 @InterfaceAudience.Private
 @InterfaceAudience.Private

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

@@ -85,11 +85,11 @@ public class ClientDatanodeProtocolTranslatorPB implements
     ProtocolTranslator, Closeable {
     ProtocolTranslator, Closeable {
   public static final Logger LOG = LoggerFactory
   public static final Logger LOG = LoggerFactory
       .getLogger(ClientDatanodeProtocolTranslatorPB.class);
       .getLogger(ClientDatanodeProtocolTranslatorPB.class);
-  
+
   /** RpcController is not used and hence is set to null */
   /** RpcController is not used and hence is set to null */
   private final static RpcController NULL_CONTROLLER = null;
   private final static RpcController NULL_CONTROLLER = null;
   private final ClientDatanodeProtocolPB rpcProxy;
   private final ClientDatanodeProtocolPB rpcProxy;
-  private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES = 
+  private final static RefreshNamenodesRequestProto VOID_REFRESH_NAMENODES =
       RefreshNamenodesRequestProto.newBuilder().build();
       RefreshNamenodesRequestProto.newBuilder().build();
   private final static GetDatanodeInfoRequestProto VOID_GET_DATANODE_INFO =
   private final static GetDatanodeInfoRequestProto VOID_GET_DATANODE_INFO =
       GetDatanodeInfoRequestProto.newBuilder().build();
       GetDatanodeInfoRequestProto.newBuilder().build();
@@ -107,16 +107,16 @@ public class ClientDatanodeProtocolTranslatorPB implements
   public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
   public ClientDatanodeProtocolTranslatorPB(DatanodeID datanodeid,
       Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
       Configuration conf, int socketTimeout, boolean connectToDnViaHostname,
       LocatedBlock locatedBlock) throws IOException {
       LocatedBlock locatedBlock) throws IOException {
-    rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf, 
-                  socketTimeout, connectToDnViaHostname, locatedBlock);
+    rpcProxy = createClientDatanodeProtocolProxy( datanodeid, conf,
+        socketTimeout, connectToDnViaHostname, locatedBlock);
   }
   }
-  
+
   public ClientDatanodeProtocolTranslatorPB(InetSocketAddress addr,
   public ClientDatanodeProtocolTranslatorPB(InetSocketAddress addr,
       UserGroupInformation ticket, Configuration conf, SocketFactory factory)
       UserGroupInformation ticket, Configuration conf, SocketFactory factory)
       throws IOException {
       throws IOException {
     rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory, 0);
     rpcProxy = createClientDatanodeProtocolProxy(addr, ticket, conf, factory, 0);
   }
   }
-  
+
   /**
   /**
    * Constructor.
    * Constructor.
    * @param datanodeid Datanode to connect to.
    * @param datanodeid Datanode to connect to.
@@ -138,7 +138,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
 
 
   static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
   static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
       DatanodeID datanodeid, Configuration conf, int socketTimeout,
       DatanodeID datanodeid, Configuration conf, int socketTimeout,
-      boolean connectToDnViaHostname, LocatedBlock locatedBlock) throws IOException {
+      boolean connectToDnViaHostname, LocatedBlock locatedBlock)
+      throws IOException {
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     final String dnAddr = datanodeid.getIpcAddr(connectToDnViaHostname);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
     InetSocketAddress addr = NetUtils.createSocketAddr(dnAddr);
     LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
     LOG.debug("Connecting to datanode {} addr={}", dnAddr, addr);
@@ -160,7 +161,7 @@ public class ClientDatanodeProtocolTranslatorPB implements
     return createClientDatanodeProtocolProxy(addr, ticket, confWithNoIpcIdle,
     return createClientDatanodeProtocolProxy(addr, ticket, confWithNoIpcIdle,
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
         NetUtils.getDefaultSocketFactory(conf), socketTimeout);
   }
   }
-  
+
   static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
   static ClientDatanodeProtocolPB createClientDatanodeProtocolProxy(
       InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
       InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
       SocketFactory factory, int socketTimeout) throws IOException {
       SocketFactory factory, int socketTimeout) throws IOException {
@@ -178,8 +179,9 @@ public class ClientDatanodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public long getReplicaVisibleLength(ExtendedBlock b) throws IOException {
   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 {
     try {
       return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
       return rpcProxy.getReplicaVisibleLength(NULL_CONTROLLER, req).getLength();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -212,8 +214,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
       Token<BlockTokenIdentifier> token) throws IOException {
       Token<BlockTokenIdentifier> token) throws IOException {
     GetBlockLocalPathInfoRequestProto req =
     GetBlockLocalPathInfoRequestProto req =
         GetBlockLocalPathInfoRequestProto.newBuilder()
         GetBlockLocalPathInfoRequestProto.newBuilder()
-        .setBlock(PBHelperClient.convert(block))
-        .setToken(PBHelperClient.convert(token)).build();
+            .setBlock(PBHelperClient.convert(block))
+            .setToken(PBHelperClient.convert(token)).build();
     GetBlockLocalPathInfoResponseProto resp;
     GetBlockLocalPathInfoResponseProto resp;
     try {
     try {
       resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
       resp = rpcProxy.getBlockLocalPathInfo(NULL_CONTROLLER, req);
@@ -251,7 +253,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
   public DatanodeLocalInfo getDatanodeInfo() throws IOException {
   public DatanodeLocalInfo getDatanodeInfo() throws IOException {
     GetDatanodeInfoResponseProto response;
     GetDatanodeInfoResponseProto response;
     try {
     try {
-      response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER, VOID_GET_DATANODE_INFO);
+      response = rpcProxy.getDatanodeInfo(NULL_CONTROLLER,
+          VOID_GET_DATANODE_INFO);
       return PBHelperClient.convert(response.getLocalInfo());
       return PBHelperClient.convert(response.getLocalInfo());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
@@ -268,7 +271,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public ReconfigurationTaskStatus getReconfigurationStatus() throws IOException {
+  public ReconfigurationTaskStatus getReconfigurationStatus()
+      throws IOException {
     GetReconfigurationStatusResponseProto response;
     GetReconfigurationStatusResponseProto response;
     Map<PropertyChange, Optional<String>> statusMap = null;
     Map<PropertyChange, Optional<String>> statusMap = null;
     long startTime;
     long startTime;
@@ -318,8 +322,8 @@ public class ClientDatanodeProtocolTranslatorPB implements
     try {
     try {
       rpcProxy.triggerBlockReport(NULL_CONTROLLER,
       rpcProxy.triggerBlockReport(NULL_CONTROLLER,
           TriggerBlockReportRequestProto.newBuilder().
           TriggerBlockReportRequestProto.newBuilder().
-            setIncremental(options.isIncremental()).
-            build());
+              setIncremental(options.isIncremental()).
+              build());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(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)
     protocolVersion = 1)
 /**
 /**
  * Protocol that a clients use to communicate with the NameNode.
  * Protocol that a clients use to communicate with the NameNode.
- * 
+ *
  * Note: This extends the protocolbuffer service based interface to
  * Note: This extends the protocolbuffer service based interface to
  * add annotations required for security.
  * add annotations required for security.
  */
  */
-public interface ClientNamenodeProtocolPB extends 
-  ClientNamenodeProtocol.BlockingInterface {
+public interface ClientNamenodeProtocolPB extends
+    ClientNamenodeProtocol.BlockingInterface {
 }
 }

+ 150 - 196
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;
 package org.apache.hadoop.hdfs.protocolPB;
 
 
 import java.io.Closeable;
 import java.io.Closeable;
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.EnumSet;
 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.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.AclEntry;
 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.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 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.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 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.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 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.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 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.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 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.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
@@ -177,8 +170,6 @@ import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.SetXAttrRequestProto;
 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.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 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.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
@@ -188,7 +179,6 @@ import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.ProtocolTranslator;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RpcClientUtil;
 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.CancelDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
@@ -213,41 +203,38 @@ public class ClientNamenodeProtocolTranslatorPB implements
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
     ProtocolMetaInterface, ClientProtocol, Closeable, ProtocolTranslator {
   final private ClientNamenodeProtocolPB rpcProxy;
   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 =
   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 = 
-  RollEditsRequestProto.getDefaultInstance();
+  private final static RollEditsRequestProto VOID_ROLLEDITS_REQUEST =
+      RollEditsRequestProto.getDefaultInstance();
 
 
   private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
   private final static RefreshNodesRequestProto VOID_REFRESH_NODES_REQUEST =
-  RefreshNodesRequestProto.newBuilder().build();
+      RefreshNodesRequestProto.newBuilder().build();
 
 
   private final static FinalizeUpgradeRequestProto
   private final static FinalizeUpgradeRequestProto
-  VOID_FINALIZE_UPGRADE_REQUEST =
+      VOID_FINALIZE_UPGRADE_REQUEST =
       FinalizeUpgradeRequestProto.newBuilder().build();
       FinalizeUpgradeRequestProto.newBuilder().build();
 
 
   private final static GetDataEncryptionKeyRequestProto
   private final static GetDataEncryptionKeyRequestProto
-  VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
+      VOID_GET_DATA_ENCRYPTIONKEY_REQUEST =
       GetDataEncryptionKeyRequestProto.newBuilder().build();
       GetDataEncryptionKeyRequestProto.newBuilder().build();
 
 
   private final static GetStoragePoliciesRequestProto
   private final static GetStoragePoliciesRequestProto
-  VOID_GET_STORAGE_POLICIES_REQUEST =
+      VOID_GET_STORAGE_POLICIES_REQUEST =
       GetStoragePoliciesRequestProto.newBuilder().build();
       GetStoragePoliciesRequestProto.newBuilder().build();
 
 
   private final static GetErasureCodingPoliciesRequestProto
   private final static GetErasureCodingPoliciesRequestProto
-  VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto
+      VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto
       .newBuilder().build();
       .newBuilder().build();
 
 
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
     rpcProxy = proxy;
   }
   }
-  
+
   @Override
   @Override
   public void close() {
   public void close() {
     RPC.stopProxy(rpcProxy);
     RPC.stopProxy(rpcProxy);
@@ -255,8 +242,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public LocatedBlocks getBlockLocations(String src, long offset, long length)
   public LocatedBlocks getBlockLocations(String src, long offset, long length)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
     GetBlockLocationsRequestProto req = GetBlockLocationsRequestProto
         .newBuilder()
         .newBuilder()
         .setSrc(src)
         .setSrc(src)
@@ -267,7 +253,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
       GetBlockLocationsResponseProto resp = rpcProxy.getBlockLocations(null,
           req);
           req);
       return resp.hasLocations() ?
       return resp.hasLocations() ?
-        PBHelperClient.convert(resp.getLocations()) : null;
+          PBHelperClient.convert(resp.getLocations()) : null;
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
@@ -287,13 +273,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   @Override
   public HdfsFileStatus create(String src, FsPermission masked,
   public HdfsFileStatus create(String src, FsPermission masked,
       String clientName, EnumSetWritable<CreateFlag> flag,
       String clientName, EnumSetWritable<CreateFlag> flag,
-      boolean createParent, short replication, long blockSize, 
+      boolean createParent, short replication, long blockSize,
       CryptoProtocolVersion[] supportedVersions)
       CryptoProtocolVersion[] supportedVersions)
-      throws AccessControlException, AlreadyBeingCreatedException,
-      DSQuotaExceededException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      throws IOException {
     CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
     CreateRequestProto.Builder builder = CreateRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setMasked(PBHelperClient.convert(masked))
         .setMasked(PBHelperClient.convert(masked))
@@ -302,7 +284,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setCreateParent(createParent)
         .setCreateParent(createParent)
         .setReplication(replication)
         .setReplication(replication)
         .setBlockSize(blockSize);
         .setBlockSize(blockSize);
-    builder.addAllCryptoProtocolVersion(PBHelperClient.convert(supportedVersions));
+    builder.addAllCryptoProtocolVersion(
+        PBHelperClient.convert(supportedVersions));
     CreateRequestProto req = builder.build();
     CreateRequestProto req = builder.build();
     try {
     try {
       CreateResponseProto res = rpcProxy.create(null, req);
       CreateResponseProto res = rpcProxy.create(null, req);
@@ -315,7 +298,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public boolean truncate(String src, long newLength, String clientName)
   public boolean truncate(String src, long newLength, String clientName)
-      throws IOException, UnresolvedLinkException {
+      throws IOException {
     TruncateRequestProto req = TruncateRequestProto.newBuilder()
     TruncateRequestProto req = TruncateRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setNewLength(newLength)
         .setNewLength(newLength)
@@ -330,18 +313,17 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public LastBlockWithStatus append(String src, String clientName,
   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)
     AppendRequestProto req = AppendRequestProto.newBuilder().setSrc(src)
-        .setClientName(clientName).setFlag(PBHelperClient.convertCreateFlag(flag))
+        .setClientName(clientName).setFlag(
+            PBHelperClient.convertCreateFlag(flag))
         .build();
         .build();
     try {
     try {
       AppendResponseProto res = rpcProxy.append(null, req);
       AppendResponseProto res = rpcProxy.append(null, req);
       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
       LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient
           .convertLocatedBlockProto(res.getBlock()) : null;
           .convertLocatedBlockProto(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);
       return new LastBlockWithStatus(lastBlock, stat);
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
@@ -350,9 +332,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public boolean setReplication(String src, short replication)
   public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      throws IOException {
     SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
     SetReplicationRequestProto req = SetReplicationRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setReplication(replication)
         .setReplication(replication)
@@ -366,8 +346,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void setPermission(String src, FsPermission permission)
   public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
     SetPermissionRequestProto req = SetPermissionRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setPermission(PBHelperClient.convert(permission))
         .setPermission(PBHelperClient.convert(permission))
@@ -381,14 +360,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void setOwner(String src, String username, String groupname)
   public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
     SetOwnerRequestProto.Builder req = SetOwnerRequestProto.newBuilder()
         .setSrc(src);
         .setSrc(src);
     if (username != null)
     if (username != null)
-        req.setUsername(username);
+      req.setUsername(username);
     if (groupname != null)
     if (groupname != null)
-        req.setGroupname(groupname);
+      req.setGroupname(groupname);
     try {
     try {
       rpcProxy.setOwner(null, req.build());
       rpcProxy.setOwner(null, req.build());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -398,28 +376,24 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void abandonBlock(ExtendedBlock b, long fileId, String src,
   public void abandonBlock(ExtendedBlock b, long fileId, String src,
-      String holder) throws AccessControlException, FileNotFoundException,
-        UnresolvedLinkException, IOException {
+      String holder) throws IOException {
     AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
     AbandonBlockRequestProto req = AbandonBlockRequestProto.newBuilder()
         .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
         .setB(PBHelperClient.convert(b)).setSrc(src).setHolder(holder)
-            .setFileId(fileId).build();
+        .setFileId(fileId).build();
     try {
     try {
       rpcProxy.abandonBlock(null, req);
       rpcProxy.abandonBlock(null, req);
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
   @Override
   @Override
   public LocatedBlock addBlock(String src, String clientName,
   public LocatedBlock addBlock(String src, String clientName,
       ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
       ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId,
-      String[] favoredNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      String[] favoredNodes) throws IOException {
     AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
     AddBlockRequestProto.Builder req = AddBlockRequestProto.newBuilder()
         .setSrc(src).setClientName(clientName).setFileId(fileId);
         .setSrc(src).setClientName(clientName).setFileId(fileId);
-    if (previous != null) 
+    if (previous != null)
       req.setPrevious(PBHelperClient.convert(previous));
       req.setPrevious(PBHelperClient.convert(previous));
     if (excludeNodes != null)
     if (excludeNodes != null)
       req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
       req.addAllExcludeNodes(PBHelperClient.convert(excludeNodes));
@@ -437,10 +411,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   @Override
   public LocatedBlock getAdditionalDatanode(String src, long fileId,
   public LocatedBlock getAdditionalDatanode(String src, long fileId,
       ExtendedBlock blk, DatanodeInfo[] existings, String[] existingStorageIDs,
       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
     GetAdditionalDatanodeRequestProto req = GetAdditionalDatanodeRequestProto
         .newBuilder()
         .newBuilder()
         .setSrc(src)
         .setSrc(src)
@@ -462,9 +434,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public boolean complete(String src, String clientName,
   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()
     CompleteRequestProto.Builder req = CompleteRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setClientName(clientName)
         .setClientName(clientName)
@@ -481,7 +451,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-        .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks)))
+        .addAllBlocks(Arrays.asList(
+            PBHelperClient.convertLocatedBlocks(blocks)))
         .build();
         .build();
     try {
     try {
       rpcProxy.reportBadBlocks(null, req);
       rpcProxy.reportBadBlocks(null, req);
@@ -491,8 +462,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public boolean rename(String src, String dst) throws UnresolvedLinkException,
-      IOException {
+  public boolean rename(String src, String dst) throws IOException {
     RenameRequestProto req = RenameRequestProto.newBuilder()
     RenameRequestProto req = RenameRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setDst(dst).build();
         .setDst(dst).build();
@@ -502,14 +472,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
 
 
   @Override
   @Override
   public void rename2(String src, String dst, Rename... options)
   public void rename2(String src, String dst, Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, IOException {
+      throws IOException {
     boolean overwrite = false;
     boolean overwrite = false;
     if (options != null) {
     if (options != null) {
       for (Rename option : options) {
       for (Rename option : options) {
@@ -531,8 +498,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public void concat(String trg, String[] srcs) throws IOException,
-      UnresolvedLinkException {
+  public void concat(String trg, String[] srcs) throws IOException {
     ConcatRequestProto req = ConcatRequestProto.newBuilder().
     ConcatRequestProto req = ConcatRequestProto.newBuilder().
         setTrg(trg).
         setTrg(trg).
         addAllSrcs(Arrays.asList(srcs)).build();
         addAllSrcs(Arrays.asList(srcs)).build();
@@ -545,10 +511,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
 
 
   @Override
   @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 {
     try {
       return rpcProxy.delete(null, req).getResult();
       return rpcProxy.delete(null, req).getResult();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -558,10 +523,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
   public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      IOException {
+      throws IOException {
     MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
     MkdirsRequestProto req = MkdirsRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setMasked(PBHelperClient.convert(masked))
         .setMasked(PBHelperClient.convert(masked))
@@ -576,15 +538,14 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public DirectoryListing getListing(String src, byte[] startAfter,
   public DirectoryListing getListing(String src, byte[] startAfter,
-      boolean needLocation) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
+      boolean needLocation) throws IOException {
     GetListingRequestProto req = GetListingRequestProto.newBuilder()
     GetListingRequestProto req = GetListingRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setStartAfter(ByteString.copyFrom(startAfter))
         .setStartAfter(ByteString.copyFrom(startAfter))
         .setNeedLocation(needLocation).build();
         .setNeedLocation(needLocation).build();
     try {
     try {
       GetListingResponseProto result = rpcProxy.getListing(null, req);
       GetListingResponseProto result = rpcProxy.getListing(null, req);
-      
+
       if (result.hasDirList()) {
       if (result.hasDirList()) {
         return PBHelperClient.convert(result.getDirList());
         return PBHelperClient.convert(result.getDirList());
       }
       }
@@ -595,8 +556,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException {
+  public void renewLease(String clientName) throws IOException {
     RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
     RenewLeaseRequestProto req = RenewLeaseRequestProto.newBuilder()
         .setClientName(clientName).build();
         .setClientName(clientName).build();
     try {
     try {
@@ -616,7 +576,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       return rpcProxy.recoverLease(null, req).getResult();
       return rpcProxy.recoverLease(null, req).getResult();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
-    }  
+    }
   }
   }
 
 
   @Override
   @Override
@@ -644,22 +604,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public DatanodeStorageReport[] getDatanodeStorageReport(DatanodeReportType type)
-      throws IOException {
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      DatanodeReportType type) throws IOException {
     final GetDatanodeStorageReportRequestProto req
     final GetDatanodeStorageReportRequestProto req
         = GetDatanodeStorageReportRequestProto.newBuilder()
         = GetDatanodeStorageReportRequestProto.newBuilder()
-            .setType(PBHelperClient.convert(type)).build();
+        .setType(PBHelperClient.convert(type)).build();
     try {
     try {
       return PBHelperClient.convertDatanodeStorageReports(
       return PBHelperClient.convertDatanodeStorageReports(
-          rpcProxy.getDatanodeStorageReport(null, req).getDatanodeStorageReportsList());
+          rpcProxy.getDatanodeStorageReport(null, req)
+              .getDatanodeStorageReportsList());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
 
 
   @Override
   @Override
-  public long getPreferredBlockSize(String filename) throws IOException,
-      UnresolvedLinkException {
+  public long getPreferredBlockSize(String filename) throws IOException {
     GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
     GetPreferredBlockSizeRequestProto req = GetPreferredBlockSizeRequestProto
         .newBuilder()
         .newBuilder()
         .setFilename(filename)
         .setFilename(filename)
@@ -672,9 +632,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException {
+  public boolean setSafeMode(SafeModeAction action, boolean isChecked)
+      throws IOException {
     SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
     SetSafeModeRequestProto req = SetSafeModeRequestProto.newBuilder()
-        .setAction(PBHelperClient.convert(action)).setChecked(isChecked).build();
+        .setAction(PBHelperClient.convert(action))
+        .setChecked(isChecked).build();
     try {
     try {
       return rpcProxy.setSafeMode(null, req).getResult();
       return rpcProxy.setSafeMode(null, req).getResult();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -692,9 +654,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
   @Override
   @Override
-  public long rollEdits() throws AccessControlException, IOException {
+  public long rollEdits() throws IOException {
     try {
     try {
       RollEditsResponseProto resp = rpcProxy.rollEdits(null,
       RollEditsResponseProto resp = rpcProxy.rollEdits(null,
           VOID_ROLLEDITS_REQUEST);
           VOID_ROLLEDITS_REQUEST);
@@ -705,8 +667,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException{
+  public boolean restoreFailedStorage(String arg) throws IOException{
     RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
     RestoreFailedStorageRequestProto req = RestoreFailedStorageRequestProto
         .newBuilder()
         .newBuilder()
         .setArg(arg).build();
         .setArg(arg).build();
@@ -736,11 +697,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+      throws IOException {
     final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
     final RollingUpgradeRequestProto r = RollingUpgradeRequestProto.newBuilder()
         .setAction(PBHelperClient.convert(action)).build();
         .setAction(PBHelperClient.convert(action)).build();
     try {
     try {
-      final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
+      final RollingUpgradeResponseProto proto =
+          rpcProxy.rollingUpgrade(null, r);
       if (proto.hasRollingUpgradeInfo()) {
       if (proto.hasRollingUpgradeInfo()) {
         return PBHelperClient.convert(proto.getRollingUpgradeInfo());
         return PBHelperClient.convert(proto.getRollingUpgradeInfo());
       }
       }
@@ -753,9 +716,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   @Override
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
   public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
       throws IOException {
       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);
       req.setCookie(cookie);
     try {
     try {
       return PBHelperClient.convert(
       return PBHelperClient.convert(
@@ -778,8 +741,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
+  public HdfsFileStatus getFileInfo(String src) throws IOException {
     GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
     GetFileInfoRequestProto req = GetFileInfoRequestProto.newBuilder()
         .setSrc(src).build();
         .setSrc(src).build();
     try {
     try {
@@ -791,23 +753,21 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException {
+  public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
     GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
     GetFileLinkInfoRequestProto req = GetFileLinkInfoRequestProto.newBuilder()
         .setSrc(src).build();
         .setSrc(src).build();
     try {
     try {
       GetFileLinkInfoResponseProto result = rpcProxy.getFileLinkInfo(null, req);
       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) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
 
 
   @Override
   @Override
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+  public ContentSummary getContentSummary(String path) throws IOException {
     GetContentSummaryRequestProto req = GetContentSummaryRequestProto
     GetContentSummaryRequestProto req = GetContentSummaryRequestProto
         .newBuilder()
         .newBuilder()
         .setPath(path)
         .setPath(path)
@@ -822,9 +782,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
-                       StorageType type)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException {
+      StorageType type) throws IOException {
     final SetQuotaRequestProto.Builder builder
     final SetQuotaRequestProto.Builder builder
         = SetQuotaRequestProto.newBuilder()
         = SetQuotaRequestProto.newBuilder()
         .setPath(path)
         .setPath(path)
@@ -843,12 +801,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void fsync(String src, long fileId, String client,
   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)
     FsyncRequestProto req = FsyncRequestProto.newBuilder().setSrc(src)
         .setClient(client).setLastBlockLength(lastBlockLength)
         .setClient(client).setLastBlockLength(lastBlockLength)
-            .setFileId(fileId).build();
+        .setFileId(fileId).build();
     try {
     try {
       rpcProxy.fsync(null, req);
       rpcProxy.fsync(null, req);
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -857,9 +813,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @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()
     SetTimesRequestProto req = SetTimesRequestProto.newBuilder()
         .setSrc(src)
         .setSrc(src)
         .setMtime(mtime)
         .setMtime(mtime)
@@ -874,10 +828,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void createSymlink(String target, String link, FsPermission dirPerm,
   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()
     CreateSymlinkRequestProto req = CreateSymlinkRequestProto.newBuilder()
         .setTarget(target)
         .setTarget(target)
         .setLink(link)
         .setLink(link)
@@ -892,8 +843,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException {
+  public String getLinkTarget(String path) throws IOException {
     GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
     GetLinkTargetRequestProto req = GetLinkTargetRequestProto.newBuilder()
         .setPath(path).build();
         .setPath(path).build();
     try {
     try {
@@ -922,7 +872,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void updatePipeline(String clientName, ExtendedBlock oldBlock,
   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()
     UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
         .setClientName(clientName)
         .setClientName(clientName)
         .setOldBlock(PBHelperClient.convert(oldBlock))
         .setOldBlock(PBHelperClient.convert(oldBlock))
@@ -945,9 +896,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setRenewer(renewer == null ? "" : renewer.toString())
         .setRenewer(renewer == null ? "" : renewer.toString())
         .build();
         .build();
     try {
     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) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
@@ -956,9 +908,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   @Override
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
   public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
       throws IOException {
-    RenewDelegationTokenRequestProto req = RenewDelegationTokenRequestProto.newBuilder().
-        setToken(PBHelperClient.convert(token)).
-        build();
+    RenewDelegationTokenRequestProto req =
+        RenewDelegationTokenRequestProto.newBuilder().
+            setToken(PBHelperClient.convert(token)).
+            build();
     try {
     try {
       return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
       return rpcProxy.renewDelegationToken(null, req).getNewExpiryTime();
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -982,9 +935,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void setBalancerBandwidth(long bandwidth) throws IOException {
   public void setBalancerBandwidth(long bandwidth) throws IOException {
-    SetBalancerBandwidthRequestProto req = SetBalancerBandwidthRequestProto.newBuilder()
-        .setBandwidth(bandwidth)
-        .build();
+    SetBalancerBandwidthRequestProto req =
+        SetBalancerBandwidthRequestProto.newBuilder()
+            .setBandwidth(bandwidth)
+            .build();
     try {
     try {
       rpcProxy.setBalancerBandwidth(null, req);
       rpcProxy.setBalancerBandwidth(null, req);
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -998,23 +952,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
         ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         ClientNamenodeProtocolPB.class, RPC.RpcKind.RPC_PROTOCOL_BUFFER,
         RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
         RPC.getProtocolVersion(ClientNamenodeProtocolPB.class), methodName);
   }
   }
-  
+
   @Override
   @Override
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
   public DataEncryptionKey getDataEncryptionKey() throws IOException {
     try {
     try {
       GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
       GetDataEncryptionKeyResponseProto rsp = rpcProxy.getDataEncryptionKey(
           null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
           null, VOID_GET_DATA_ENCRYPTIONKEY_REQUEST);
-     return rsp.hasDataEncryptionKey() ? 
+      return rsp.hasDataEncryptionKey() ?
           PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
           PBHelperClient.convert(rsp.getDataEncryptionKey()) : null;
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
 
 
   @Override
   @Override
-  public boolean isFileClosed(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException {
+  public boolean isFileClosed(String src) throws IOException {
     IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
     IsFileClosedRequestProto req = IsFileClosedRequestProto.newBuilder()
         .setSrc(src).build();
         .setSrc(src).build();
     try {
     try {
@@ -1044,7 +997,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
   @Override
   @Override
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
       throws IOException {
@@ -1056,7 +1009,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
   @Override
   @Override
   public void allowSnapshot(String snapshotRoot) throws IOException {
   public void allowSnapshot(String snapshotRoot) throws IOException {
     AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
     AllowSnapshotRequestProto req = AllowSnapshotRequestProto.newBuilder()
@@ -1095,12 +1048,12 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   @Override
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
       throws IOException {
-    GetSnapshottableDirListingRequestProto req = 
+    GetSnapshottableDirListingRequestProto req =
         GetSnapshottableDirListingRequestProto.newBuilder().build();
         GetSnapshottableDirListingRequestProto.newBuilder().build();
     try {
     try {
       GetSnapshottableDirListingResponseProto result = rpcProxy
       GetSnapshottableDirListingResponseProto result = rpcProxy
           .getSnapshottableDirListing(null, req);
           .getSnapshottableDirListing(null, req);
-      
+
       if (result.hasSnapshottableDirList()) {
       if (result.hasSnapshottableDirList()) {
         return PBHelperClient.convert(result.getSnapshottableDirList());
         return PBHelperClient.convert(result.getSnapshottableDirList());
       }
       }
@@ -1117,9 +1070,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .newBuilder().setSnapshotRoot(snapshotRoot)
         .newBuilder().setSnapshotRoot(snapshotRoot)
         .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
         .setFromSnapshot(fromSnapshot).setToSnapshot(toSnapshot).build();
     try {
     try {
-      GetSnapshotDiffReportResponseProto result = 
+      GetSnapshotDiffReportResponseProto result =
           rpcProxy.getSnapshotDiffReport(null, req);
           rpcProxy.getSnapshotDiffReport(null, req);
-    
+
       return PBHelperClient.convert(result.getDiffReport());
       return PBHelperClient.convert(result.getDiffReport());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
@@ -1188,7 +1141,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     public int size() {
     public int size() {
       return response.getElementsCount();
       return response.getElementsCount();
     }
     }
-    
+
     @Override
     @Override
     public boolean hasMore() {
     public boolean hasMore() {
       return response.getHasMore();
       return response.getHasMore();
@@ -1196,19 +1149,18 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public BatchedEntries<CacheDirectiveEntry>
-      listCacheDirectives(long prevId,
-          CacheDirectiveInfo filter) throws IOException {
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(long prevId,
+      CacheDirectiveInfo filter) throws IOException {
     if (filter == null) {
     if (filter == null) {
       filter = new CacheDirectiveInfo.Builder().build();
       filter = new CacheDirectiveInfo.Builder().build();
     }
     }
     try {
     try {
       return new BatchedCacheEntries(
       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) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
@@ -1216,7 +1168,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void addCachePool(CachePoolInfo info) throws IOException {
   public void addCachePool(CachePoolInfo info) throws IOException {
-    AddCachePoolRequestProto.Builder builder = 
+    AddCachePoolRequestProto.Builder builder =
         AddCachePoolRequestProto.newBuilder();
         AddCachePoolRequestProto.newBuilder();
     builder.setInfo(PBHelperClient.convert(info));
     builder.setInfo(PBHelperClient.convert(info));
     try {
     try {
@@ -1228,7 +1180,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void modifyCachePool(CachePoolInfo req) throws IOException {
   public void modifyCachePool(CachePoolInfo req) throws IOException {
-    ModifyCachePoolRequestProto.Builder builder = 
+    ModifyCachePoolRequestProto.Builder builder =
         ModifyCachePoolRequestProto.newBuilder();
         ModifyCachePoolRequestProto.newBuilder();
     builder.setInfo(PBHelperClient.convert(req));
     builder.setInfo(PBHelperClient.convert(req));
     try {
     try {
@@ -1241,22 +1193,22 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   @Override
   public void removeCachePool(String cachePoolName) throws IOException {
   public void removeCachePool(String cachePoolName) throws IOException {
     try {
     try {
-      rpcProxy.removeCachePool(null, 
+      rpcProxy.removeCachePool(null,
           RemoveCachePoolRequestProto.newBuilder().
           RemoveCachePoolRequestProto.newBuilder().
-            setPoolName(cachePoolName).build());
+              setPoolName(cachePoolName).build());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
 
 
   private static class BatchedCachePoolEntries
   private static class BatchedCachePoolEntries
-    implements BatchedEntries<CachePoolEntry> {
-      private final ListCachePoolsResponseProto proto;
-    
+      implements BatchedEntries<CachePoolEntry> {
+    private final ListCachePoolsResponseProto proto;
+
     public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
     public BatchedCachePoolEntries(ListCachePoolsResponseProto proto) {
       this.proto = proto;
       this.proto = proto;
     }
     }
-      
+
     @Override
     @Override
     public CachePoolEntry get(int i) {
     public CachePoolEntry get(int i) {
       CachePoolEntryProto elem = proto.getEntries(i);
       CachePoolEntryProto elem = proto.getEntries(i);
@@ -1267,7 +1219,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     public int size() {
     public int size() {
       return proto.getEntriesCount();
       return proto.getEntriesCount();
     }
     }
-    
+
     @Override
     @Override
     public boolean hasMore() {
     public boolean hasMore() {
       return proto.getHasMore();
       return proto.getHasMore();
@@ -1279,9 +1231,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throws IOException {
       throws IOException {
     try {
     try {
       return new BatchedCachePoolEntries(
       return new BatchedCachePoolEntries(
-        rpcProxy.listCachePools(null,
-          ListCachePoolsRequestProto.newBuilder().
-            setPrevPoolName(prevKey).build()));
+          rpcProxy.listCachePools(null,
+              ListCachePoolsRequestProto.newBuilder().
+                  setPrevPoolName(prevKey).build()));
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
@@ -1361,9 +1313,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
 
 
   @Override
   @Override
   public void createEncryptionZone(String src, String keyName)
   public void createEncryptionZone(String src, String keyName)
-    throws IOException {
+      throws IOException {
     final CreateEncryptionZoneRequestProto.Builder builder =
     final CreateEncryptionZoneRequestProto.Builder builder =
-      CreateEncryptionZoneRequestProto.newBuilder();
+        CreateEncryptionZoneRequestProto.newBuilder();
     builder.setSrc(src);
     builder.setSrc(src);
     if (keyName != null && !keyName.isEmpty()) {
     if (keyName != null && !keyName.isEmpty()) {
       builder.setKeyName(keyName);
       builder.setKeyName(keyName);
@@ -1377,8 +1329,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public EncryptionZone getEZForPath(String src)
-      throws IOException {
+  public EncryptionZone getEZForPath(String src) throws IOException {
     final GetEZForPathRequestProto.Builder builder =
     final GetEZForPathRequestProto.Builder builder =
         GetEZForPathRequestProto.newBuilder();
         GetEZForPathRequestProto.newBuilder();
     builder.setSrc(src);
     builder.setSrc(src);
@@ -1400,9 +1351,9 @@ public class ClientNamenodeProtocolTranslatorPB implements
   public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
   public BatchedEntries<EncryptionZone> listEncryptionZones(long id)
       throws IOException {
       throws IOException {
     final ListEncryptionZonesRequestProto req =
     final ListEncryptionZonesRequestProto req =
-      ListEncryptionZonesRequestProto.newBuilder()
-          .setId(id)
-          .build();
+        ListEncryptionZonesRequestProto.newBuilder()
+            .setId(id)
+            .build();
     try {
     try {
       EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
       EncryptionZonesProtos.ListEncryptionZonesResponseProto response =
           rpcProxy.listEncryptionZones(null, req);
           rpcProxy.listEncryptionZones(null, req);
@@ -1411,8 +1362,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       for (EncryptionZoneProto p : response.getZonesList()) {
       for (EncryptionZoneProto p : response.getZonesList()) {
         elements.add(PBHelperClient.convert(p));
         elements.add(PBHelperClient.convert(p));
       }
       }
-      return new BatchedListEntries<EncryptionZone>(elements,
-          response.getHasMore());
+      return new BatchedListEntries<>(elements, response.getHasMore());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
@@ -1449,7 +1399,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
   @Override
   @Override
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
   public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs)
       throws IOException {
       throws IOException {
@@ -1465,11 +1415,11 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }
   }
   }
-  
+
   @Override
   @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);
     builder.setSrc(src);
     ListXAttrsRequestProto req = builder.build();
     ListXAttrsRequestProto req = builder.build();
     try {
     try {
@@ -1566,8 +1516,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
       ErasureCodingPolicy[] ecPolicies =
       ErasureCodingPolicy[] ecPolicies =
           new ErasureCodingPolicy[response.getEcPoliciesCount()];
           new ErasureCodingPolicy[response.getEcPoliciesCount()];
       int i = 0;
       int i = 0;
-      for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) {
-        ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
+      for (ErasureCodingPolicyProto ecPolicyProto :
+          response.getEcPoliciesList()) {
+        ecPolicies[i++] =
+            PBHelperClient.convertErasureCodingPolicy(ecPolicyProto);
       }
       }
       return ecPolicies;
       return ecPolicies;
     } catch (ServiceException e) {
     } catch (ServiceException e) {
@@ -1576,14 +1528,16 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
   }
 
 
   @Override
   @Override
-  public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException {
-    GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder()
-        .setSrc(src).build();
+  public ErasureCodingPolicy getErasureCodingPolicy(String src)
+      throws IOException {
+    GetErasureCodingPolicyRequestProto req =
+        GetErasureCodingPolicyRequestProto.newBuilder().setSrc(src).build();
     try {
     try {
-      GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy(
-          null, req);
+      GetErasureCodingPolicyResponseProto response =
+          rpcProxy.getErasureCodingPolicy(null, req);
       if (response.hasEcPolicy()) {
       if (response.hasEcPolicy()) {
-        return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy());
+        return PBHelperClient.convertErasureCodingPolicy(
+            response.getEcPolicy());
       }
       }
       return null;
       return null;
     } catch (ServiceException e) {
     } catch (ServiceException e) {

File diff suppressed because it is too large
+ 334 - 330
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java


+ 11 - 9
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
 @InterfaceAudience.Private
 public class DelegationTokenIdentifier
 public class DelegationTokenIdentifier
     extends AbstractDelegationTokenIdentifier {
     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.
    * Create an empty delegation token identifier for reading into.
@@ -88,13 +89,14 @@ public class DelegationTokenIdentifier
     }
     }
   }
   }
 
 
-  public static class SWebHdfsDelegationTokenIdentifier extends WebHdfsDelegationTokenIdentifier {
-	public SWebHdfsDelegationTokenIdentifier() {
-	  super();
-	}
-	@Override
-	public Text getKind() {
-	  return WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
-	}
+  public static class SWebHdfsDelegationTokenIdentifier
+      extends WebHdfsDelegationTokenIdentifier {
+    public SWebHdfsDelegationTokenIdentifier() {
+      super();
+    }
+    @Override
+    public Text getKind() {
+      return WebHdfsConstants.SWEBHDFS_TOKEN_KIND;
+    }
   }
   }
 }
 }

+ 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
    * 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
    * This method should only be called by non-hdfs filesystems that do not
    * use the rpc port to acquire tokens.  Ex. webhdfs
    * use the rpc port to acquire tokens.  Ex. webhdfs
    * @param nnUri of the remote namenode
    * @param nnUri of the remote namenode
@@ -56,15 +56,15 @@ public class DelegationTokenSelector
     // and correctly determine the value
     // and correctly determine the value
     Text serviceName = SecurityUtil.buildTokenService(nnUri);
     Text serviceName = SecurityUtil.buildTokenService(nnUri);
     final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
     final String nnServiceName = conf.get(SERVICE_NAME_KEY + serviceName);
-    
+
     int nnRpcPort = HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
     int nnRpcPort = HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT;
     if (nnServiceName != null) {
     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
     // use original hostname from the uri to avoid unintentional host resolving
     serviceName = SecurityUtil.buildTokenService(
     serviceName = SecurityUtil.buildTokenService(
     		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
     		NetUtils.createSocketAddrForHost(nnUri.getHost(), nnRpcPort));
-    
+
     return selectToken(serviceName, tokens);
     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);
       BlockMetadataHeader.class);
 
 
   public static final short VERSION = 1;
   public static final short VERSION = 1;
-  
+
   /**
   /**
    * Header includes everything except the checksum(s) themselves.
    * Header includes everything except the checksum(s) themselves.
    * Version is two bytes. Following it is the DataChecksum
    * Version is two bytes. Following it is the DataChecksum
-   * that occupies 5 bytes. 
+   * that occupies 5 bytes.
    */
    */
   private final short version;
   private final short version;
   private DataChecksum checksum = null;
   private DataChecksum checksum = null;
@@ -66,7 +66,7 @@ public class BlockMetadataHeader {
     this.checksum = checksum;
     this.checksum = checksum;
     this.version = version;
     this.version = version;
   }
   }
-  
+
   /** Get the version */
   /** Get the version */
   public short getVersion() {
   public short getVersion() {
     return version;
     return version;
@@ -137,13 +137,14 @@ public class BlockMetadataHeader {
    * @return Metadata Header
    * @return Metadata Header
    * @throws IOException
    * @throws IOException
    */
    */
-  public static BlockMetadataHeader readHeader(DataInputStream in) throws IOException {
+  public static BlockMetadataHeader readHeader(DataInputStream in)
+      throws IOException {
     return readHeader(in.readShort(), in);
     return readHeader(in.readShort(), in);
   }
   }
-  
+
   /**
   /**
    * Reads header at the top of metadata file and returns the header.
    * Reads header at the top of metadata file and returns the header.
-   * 
+   *
    * @return metadata header for the block
    * @return metadata header for the block
    * @throws IOException
    * @throws IOException
    */
    */
@@ -157,39 +158,40 @@ public class BlockMetadataHeader {
       IOUtils.closeStream(in);
       IOUtils.closeStream(in);
     }
     }
   }
   }
-  
+
   /**
   /**
    * Read the header at the beginning of the given block meta file.
    * Read the header at the beginning of the given block meta file.
    * The current file position will be altered by this method.
    * The current file position will be altered by this method.
    * If an error occurs, the file is <em>not</em> closed.
    * 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()];
     byte[] buf = new byte[getHeaderSize()];
     raf.seek(0);
     raf.seek(0);
     raf.readFully(buf, 0, buf.length);
     raf.readFully(buf, 0, buf.length);
     return readHeader(new DataInputStream(new ByteArrayInputStream(buf)));
     return readHeader(new DataInputStream(new ByteArrayInputStream(buf)));
   }
   }
-  
+
   // Version is already read.
   // 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);
     DataChecksum checksum = DataChecksum.newDataChecksum(in);
     return new BlockMetadataHeader(version, checksum);
     return new BlockMetadataHeader(version, checksum);
   }
   }
-  
+
   /**
   /**
    * This writes all the fields till the beginning of checksum.
    * This writes all the fields till the beginning of checksum.
    * @param out DataOutputStream
    * @param out DataOutputStream
    * @throws IOException
    * @throws IOException
    */
    */
   @VisibleForTesting
   @VisibleForTesting
-  public static void writeHeader(DataOutputStream out, 
-                                  BlockMetadataHeader header) 
+  public static void writeHeader(DataOutputStream out,
+                                  BlockMetadataHeader header)
                                   throws IOException {
                                   throws IOException {
     out.writeShort(header.getVersion());
     out.writeShort(header.getVersion());
     header.getChecksum().writeHeader(out);
     header.getChecksum().writeHeader(out);
   }
   }
-  
+
   /**
   /**
    * Writes all the fields till the beginning of checksum.
    * Writes all the fields till the beginning of checksum.
    * @throws IOException on error
    * @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 {
 public class CachingStrategy {
   private final Boolean dropBehind; // null = use server defaults
   private final Boolean dropBehind; // null = use server defaults
   private final Long readahead; // null = use server defaults
   private final Long readahead; // null = use server defaults
-  
+
   public static CachingStrategy newDefaultStrategy() {
   public static CachingStrategy newDefaultStrategy() {
     return new CachingStrategy(null, null);
     return new CachingStrategy(null, null);
   }
   }
@@ -64,7 +64,7 @@ public class CachingStrategy {
   public Boolean getDropBehind() {
   public Boolean getDropBehind() {
     return dropBehind;
     return dropBehind;
   }
   }
-  
+
   public Long getReadahead() {
   public Long getReadahead() {
     return readahead;
     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
  * Exception indicating that DataNode does not have a replica
- * that matches the target block.  
+ * that matches the target block.
  */
  */
 public class ReplicaNotFoundException extends IOException {
 public class ReplicaNotFoundException extends IOException {
   private static final long serialVersionUID = 1L;
   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 =
   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 =
   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() {
   public ReplicaNotFoundException() {
     super();
     super();
@@ -46,7 +48,7 @@ public class ReplicaNotFoundException extends IOException {
   public ReplicaNotFoundException(ExtendedBlock b) {
   public ReplicaNotFoundException(ExtendedBlock b) {
     super("Replica not found for " + b);
     super("Replica not found for " + b);
   }
   }
-  
+
   public ReplicaNotFoundException(String msg) {
   public ReplicaNotFoundException(String msg) {
     super(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.
  * 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
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
@@ -35,4 +35,4 @@ public class SafeModeException extends IOException {
   public SafeModeException(String msg) {
   public SafeModeException(String msg) {
     super(msg);
     super(msg);
   }
   }
-}
+}

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