Browse Source

HDFS-6274. Merge r1589506 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1589507 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 11 years ago
parent
commit
a27fda9c85
81 changed files with 153 additions and 266 deletions
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java
  4. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
  5. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  6. 5 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
  7. 5 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  8. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  9. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java
  10. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/DataTransferProtocol.java
  12. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
  13. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java
  14. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  15. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  16. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  17. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  18. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  19. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  20. 1 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  21. 2 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
  24. 7 10
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  25. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
  26. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  27. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java
  28. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java
  30. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java
  31. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java
  32. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java
  33. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java
  34. 2 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java
  35. 4 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
  37. 3 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
  38. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MappableBlock.java
  39. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  40. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
  41. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachedBlock.java
  42. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java
  43. 0 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  44. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
  45. 8 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  46. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  47. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  48. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  49. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  50. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  51. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  52. 17 22
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  53. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
  54. 3 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  55. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  56. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LogsPurgeable.java
  57. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  58. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  59. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  60. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  61. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  62. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/HAState.java
  63. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java
  64. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockIdCommand.java
  65. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
  66. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java
  67. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/ServerCommand.java
  68. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
  69. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java
  70. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/HDFSConcat.java
  71. 1 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/JMXGet.java
  72. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java
  73. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DataTransferThrottler.java
  74. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java
  75. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BenchmarkThroughput.java
  76. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  77. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
  78. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
  79. 3 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
  80. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  81. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

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

@@ -122,6 +122,8 @@ Release 2.5.0 - UNRELEASED
     HDFS-6213. TestDataNodeConfig failing on Jenkins runs due to DN web port
     in use. (wang)
 
+    HDFS-6274. Cleanup javadoc warnings in HDFS code. (suresh)
+
 Release 2.4.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/Hdfs.java

@@ -67,9 +67,8 @@ public class Hdfs extends AbstractFileSystem {
    * This constructor has the signature needed by
    * {@link AbstractFileSystem#createFileSystem(URI, Configuration)}
    * 
-   * @param theUri
-   *          which must be that of Hdfs
-   * @param conf
+   * @param theUri which must be that of Hdfs
+   * @param conf configuration
    * @throws IOException
    */
   Hdfs(final URI theUri, final Configuration conf) throws IOException, URISyntaxException {

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

@@ -182,7 +182,7 @@ class BlockReaderLocal implements BlockReader {
 
   /**
    * Maximum amount of readahead we'll do.  This will always be at least the,
-   * 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
    * to do checksumming.
    * 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java

@@ -191,7 +191,8 @@ class BlockStorageLocationUtil {
   
   /**
    * Group the per-replica {@link VolumeId} info returned from
-   * {@link DFSClient#queryDatanodesForHdfsBlocksMetadata(Map)} to be associated
+   * {@link DFSClient#queryDatanodesForHdfsBlocksMetadata(Map)} to be
+   * associated
    * with the corresponding {@link LocatedBlock}.
    * 
    * @param blocks

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -1624,7 +1624,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
   /**
    * Set replication for an existing file.
    * @param src file name
-   * @param replication
+   * @param replication replication to set the file to
    * 
    * @see ClientProtocol#setReplication(String, short)
    */
@@ -2134,7 +2134,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
   /**
    * Set permissions to a file or directory.
    * @param src path name.
-   * @param permission
+   * @param permission permission to set to
    * 
    * @see ClientProtocol#setPermission(String, FsPermission)
    */

+ 5 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -389,7 +389,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Get block at the specified position.
    * Fetch it from the namenode if not cached.
    * 
-   * @param offset
+   * @param offset block corresponding to this offset in file is returned
    * @param updatePosition whether to update current position
    * @return located block
    * @throws IOException
@@ -453,14 +453,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Get blocks in the specified range.
    * Fetch them from the namenode if not cached. This function
    * will not get a read request beyond the EOF.
-   * @param offset
-   * @param length
+   * @param offset starting offset in file
+   * @param length length of data
    * @return consequent segment of located blocks
    * @throws IOException
    */
-  private synchronized List<LocatedBlock> getBlockRange(long offset, 
-                                                        long length) 
-                                                      throws IOException {
+  private synchronized List<LocatedBlock> getBlockRange(long offset,
+      long length)  throws IOException {
     // getFileLength(): returns total file length
     // locatedBlocks.getFileLength(): returns length of completed blocks
     if (offset >= getFileLength()) {
@@ -847,7 +846,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
 
   /**
    * Add corrupted block replica into map.
-   * @param corruptedBlockMap 
    */
   private void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node, 
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
@@ -1091,14 +1089,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * int, Map)} except we start up a second, parallel, 'hedged' read
    * if the first read is taking longer than configured amount of
    * time.  We then wait on which ever read returns first.
-   * 
-   * @param block
-   * @param start
-   * @param end
-   * @param buf
-   * @param offset
-   * @param corruptedBlockMap
-   * @throws IOException
    */
   private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
       long end, byte[] buf, int offset,

+ 5 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -289,9 +289,7 @@ public class DFSUtil {
    * <p>
    * Note that some components are only reserved under certain directories, e.g.
    * "/.reserved" is reserved, while "/hadoop/.reserved" is not.
-   * 
-   * @param component
-   * @return if the component is reserved
+   * @return true, if the component is reserved
    */
   public static boolean isReservedPathComponent(String component) {
     for (String reserved : HdfsConstants.RESERVED_PATH_COMPONENTS) {
@@ -1016,8 +1014,8 @@ public class DFSUtil {
   /**
    * return server http or https address from the configuration for a
    * given namenode rpc address.
-   * @param conf
    * @param namenodeAddr - namenode RPC address
+   * @param conf configuration
    * @param scheme - the scheme (http / https)
    * @return server http or https address
    * @throws IOException 
@@ -1328,7 +1326,7 @@ public class DFSUtil {
   /**
    * For given set of {@code keys} adds nameservice Id and or namenode Id
    * and returns {nameserviceId, namenodeId} when address match is found.
-   * @see #getSuffixIDs(Configuration, String, AddressMatcher)
+   * @see #getSuffixIDs(Configuration, String, String, String, AddressMatcher)
    */
   static String[] getSuffixIDs(final Configuration conf,
       final InetSocketAddress address, final String... keys) {
@@ -1500,9 +1498,8 @@ public class DFSUtil {
   /**
    * Get SPNEGO keytab Key from configuration
    * 
-   * @param conf
-   *          Configuration
-   * @param defaultKey
+   * @param conf Configuration
+   * @param defaultKey default key to be used for config lookup
    * @return DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY if the key is not empty
    *         else return defaultKey
    */

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

@@ -1477,7 +1477,7 @@ public class DistributedFileSystem extends FileSystem {
    * Get the difference between two snapshots, or between a snapshot and the
    * current tree of a directory.
    * 
-   * @see DFSClient#getSnapshotDiffReport(Path, String, String)
+   * @see DFSClient#getSnapshotDiffReport(String, String, String)
    */
   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
       final String fromSnapshot, final String toSnapshot) throws IOException {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientDatanodeProtocol.java

@@ -112,8 +112,9 @@ public interface ClientDatanodeProtocol {
   
   /**
    * Retrieves volume location information about a list of blocks on a datanode.
-   * This is in the form of an opaque {@link VolumeId} for each configured
-   * data directory, which is not guaranteed to be the same across DN restarts.
+   * This is in the form of an opaque {@link org.apache.hadoop.fs.VolumeId}
+   * for each configured data directory, which is not guaranteed to be
+   * the same across DN restarts.
    * 
    * @param blockPoolId the pool to query
    * @param blockIds

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

@@ -268,7 +268,7 @@ public interface ClientProtocol {
   /**
    * Set Owner of a path (i.e. a file or a directory).
    * The parameters username and groupname cannot both be null.
-   * @param src
+   * @param src file path
    * @param username If it is null, the original username remains unchanged.
    * @param groupname If it is null, the original groupname remains unchanged.
    *
@@ -1126,7 +1126,6 @@ public interface ClientProtocol {
   /**
    * Modify a CacheDirective in the CacheManager.
    * 
-   * @return directive The directive to modify. Must contain a directive ID.
    * @param flags {@link CacheFlag}s to use for this operation.
    * @throws IOException if the directive could not be modified
    */

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

@@ -136,7 +136,7 @@ public interface DataTransferProtocol {
   /**
    * Request a short circuit shared memory area from a DataNode.
    * 
-   * @pram clientName       The name of the client.
+   * @param clientName       The name of the client.
    */
   public void requestShortCircuitShm(String clientName) throws IOException;
   

+ 4 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java

@@ -101,9 +101,9 @@ public class BlockTokenSecretManager extends
    * 
    * @param keyUpdateInterval how often a new key will be generated
    * @param tokenLifetime how long an individual token is valid
-   * @param isHaEnabled whether or not HA is enabled
-   * @param thisNnId the NN ID of this NN in an HA setup
-   * @param otherNnId the NN ID of the other NN in an HA setup
+   * @param nnIndex namenode index
+   * @param blockPoolId block pool ID
+   * @param encryptionAlgorithm encryption algorithm to use
    */
   public BlockTokenSecretManager(long keyUpdateInterval,
       long tokenLifetime, int nnIndex, String blockPoolId,
@@ -412,8 +412,7 @@ public class BlockTokenSecretManager extends
    * @param keyId identifier of the secret key used to generate the encryption key.
    * @param nonce random value used to create the encryption key
    * @return the encryption key which corresponds to this (keyId, blockPoolId, nonce)
-   * @throws InvalidToken
-   * @throws InvalidEncryptionKeyException 
+   * @throws InvalidEncryptionKeyException
    */
   public byte[] retrieveDataEncryptionKey(int keyId, byte[] nonce)
       throws InvalidEncryptionKeyException {

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSecretManager.java

@@ -299,7 +299,7 @@ public class DelegationTokenSecretManager
    * Update the token cache with renewal record in edit logs.
    * 
    * @param identifier DelegationTokenIdentifier of the renewed token
-   * @param expiryTime
+   * @param expiryTime expirty time in milliseconds
    * @throws IOException
    */
   public synchronized void updatePersistedTokenRenewal(
@@ -429,8 +429,7 @@ public class DelegationTokenSecretManager
 
     /**
      * Private helper method to load delegation keys from fsimage.
-     * @param in
-     * @throws IOException
+     * @throws IOException on error
      */
     private synchronized void loadAllKeys(DataInput in) throws IOException {
       StartupProgress prog = NameNode.getStartupProgress();

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -242,7 +242,6 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
 
   /**
    * Find specified DatanodeDescriptor.
-   * @param dn
    * @return index or -1 if not found.
    */
   int findDatanode(DatanodeDescriptor dn) {
@@ -258,7 +257,6 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   }
   /**
    * Find specified DatanodeStorageInfo.
-   * @param dn
    * @return index or -1 if not found.
    */
   int findStorageInfo(DatanodeInfo dn) {
@@ -275,7 +273,6 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
   
   /**
    * Find specified DatanodeStorageInfo.
-   * @param storageInfo
    * @return index or -1 if not found.
    */
   int findStorageInfo(DatanodeStorageInfo storageInfo) {

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -549,7 +549,6 @@ public class BlockManager {
   }
 
   /**
-   * @param block
    * @return true if the block has minimum replicas
    */
   public boolean checkMinReplication(Block block) {
@@ -3369,7 +3368,6 @@ public class BlockManager {
    * heartbeat.
    * 
    * @return number of blocks scheduled for replication or removal.
-   * @throws IOException
    */
   int computeDatanodeWork() {
     // Blocks should not be replicated or removed if in safe mode.

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

@@ -61,7 +61,7 @@ public abstract class BlockPlacementPolicy {
    * @param srcPath the file to which this chooseTargets is being invoked.
    * @param numOfReplicas additional number of replicas wanted.
    * @param writer the writer's machine, null if not in the cluster.
-   * @param chosenNodes datanodes that have been chosen as targets.
+   * @param chosen datanodes that have been chosen as targets.
    * @param returnChosenNodes decide if the chosenNodes are returned.
    * @param excludedNodes datanodes that should not be considered as targets.
    * @param blocksize size of the data to be written.
@@ -78,8 +78,8 @@ public abstract class BlockPlacementPolicy {
                                              StorageType storageType);
   
   /**
-   * Same as {@link #chooseTarget(String, int, Node, List, boolean, 
-   * Set, long)} with added parameter {@code favoredDatanodes}
+   * Same as {@link #chooseTarget(String, int, Node, Set, long, List, StorageType)}
+   * with added parameter {@code favoredDatanodes}
    * @param favoredNodes datanodes that should be favored as targets. This
    *          is only a hint and due to cluster state, namenode may not be 
    *          able to place the blocks on these datanodes.
@@ -143,7 +143,8 @@ public abstract class BlockPlacementPolicy {
     
   /**
    * Get an instance of the configured Block Placement Policy based on the
-   * the configuration property {@link DFS_BLOCK_REPLICATOR_CLASSNAME_KEY}.
+   * the configuration property
+   * {@link  DFSConfigKeys#DFS_BLOCK_REPLICATOR_CLASSNAME_KEY}.
    * 
    * @param conf the configuration to be used
    * @param stats an object that is used to retrieve the load on the cluster
@@ -195,7 +196,6 @@ public abstract class BlockPlacementPolicy {
 
   /**
    * Get rack string from a data node
-   * @param datanode
    * @return rack of data node
    */
   protected String getRack(final DatanodeInfo datanode) {
@@ -206,7 +206,7 @@ public abstract class BlockPlacementPolicy {
    * Split data nodes into two sets, one set includes nodes on rack with
    * more than one  replica, the other set contains the remaining nodes.
    * 
-   * @param dataNodes
+   * @param dataNodes datanodes to be split into two sets
    * @param rackMap a map from rack to datanodes
    * @param moreThanOne contains nodes on rack with more than one replica
    * @param exactlyOne remains contains the remaining nodes

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

@@ -435,9 +435,9 @@ public class DatanodeManager {
   }
 
   /**
-   * Get data node by storage ID.
+   * Get data node by datanode ID.
    * 
-   * @param nodeID
+   * @param nodeID datanode ID
    * @return DatanodeDescriptor or null if the node is not found.
    * @throws UnregisteredNodeException
    */

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -831,10 +831,10 @@ public abstract class Storage extends StorageInfo {
   }
 
   /**
-   * Checks if the upgrade from the given old version is supported. If
-   * no upgrade is supported, it throws IncorrectVersionException.
-   * 
-   * @param oldVersion
+   * Checks if the upgrade from {@code oldVersion} is supported.
+   * @param oldVersion the version of the metadata to check with the current
+   *                   version
+   * @throws IOException if upgrade is not supported
    */
   public static void checkVersionUpgradable(int oldVersion) 
                                      throws IOException {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java

@@ -123,8 +123,8 @@ public class StorageInfo {
    * Get common storage fields.
    * Should be overloaded if additional fields need to be get.
    * 
-   * @param props
-   * @throws IOException
+   * @param props properties
+   * @throws IOException on error
    */
   protected void setFieldsFromProperties(
       Properties props, StorageDirectory sd) throws IOException {

+ 1 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -314,9 +314,7 @@ class BPServiceActor implements Runnable {
   }
 
   /**
-   * Retrieve the incremental BR state for a given storage UUID
-   * @param storageUuid
-   * @return
+   * @return pending incremental block report for given {@code storage}
    */
   private PerStoragePendingIncrementalBR getIncrementalBRMapForStorage(
       DatanodeStorage storage) {
@@ -339,8 +337,6 @@ class BPServiceActor implements Runnable {
    * exists for the same block it is removed.
    *
    * Caller must synchronize access using pendingIncrementalBRperStorage.
-   * @param bInfo
-   * @param storageUuid
    */
   void addPendingReplicationBlockInfo(ReceivedDeletedBlockInfo bInfo,
       DatanodeStorage storage) {

+ 2 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockMetadataHeader.java

@@ -98,7 +98,6 @@ public class BlockMetadataHeader {
 
   /**
    * This reads all the fields till the beginning of checksum.
-   * @param in 
    * @return Metadata Header
    * @throws IOException
    */
@@ -109,9 +108,7 @@ public class BlockMetadataHeader {
   /**
    * Reads header at the top of metadata file and returns the header.
    * 
-   * @param dataset
-   * @param block
-   * @return
+   * @return metadata header for the block
    * @throws IOException
    */
   public static BlockMetadataHeader readHeader(File file) throws IOException {
@@ -147,8 +144,6 @@ public class BlockMetadataHeader {
   /**
    * This writes all the fields till the beginning of checksum.
    * @param out DataOutputStream
-   * @param header 
-   * @return 
    * @throws IOException
    */
   @VisibleForTesting
@@ -161,9 +156,7 @@ public class BlockMetadataHeader {
   
   /**
    * Writes all the fields till the beginning of checksum.
-   * @param out
-   * @param checksum
-   * @throws IOException
+   * @throws IOException on error
    */
   static void writeHeader(DataOutputStream out, DataChecksum checksum)
                          throws IOException {

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -956,9 +956,9 @@ class BlockReceiver implements Closeable {
     
     /**
      * enqueue the seqno that is still be to acked by the downstream datanode.
-     * @param seqno
-     * @param lastPacketInBlock
-     * @param offsetInBlock
+     * @param seqno sequence number of the packet
+     * @param lastPacketInBlock if true, this is the last packet in block
+     * @param offsetInBlock offset of this packet in block
      */
     void enqueue(final long seqno, final boolean lastPacketInBlock,
         final long offsetInBlock, final Status ackStatus) {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -168,7 +168,7 @@ class BlockSender implements java.io.Closeable {
    * @param block Block that is being read
    * @param startOffset starting offset to read from
    * @param length length of data to read
-   * @param corruptChecksumOk
+   * @param corruptChecksumOk if true, corrupt checksum is okay
    * @param verifyChecksum verify checksum while reading the data
    * @param sendChecksum send checksum to client.
    * @param datanode datanode from which the block is being read

+ 7 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -653,7 +653,6 @@ public class DataNode extends Configured
   
   /**
    * Return the BPOfferService instance corresponding to the given block.
-   * @param block
    * @return the BPOS
    * @throws IOException if no such BPOS can be found
    */
@@ -818,9 +817,7 @@ public class DataNode extends Configured
   /**
    * After the block pool has contacted the NN, registers that block pool
    * with the secret manager, updating it with the secrets provided by the NN.
-   * @param bpRegistration
-   * @param blockPoolId
-   * @throws IOException
+   * @throws IOException on error
    */
   private synchronized void registerBlockPoolWithSecretManager(
       DatanodeRegistration bpRegistration, String blockPoolId) throws IOException {
@@ -988,9 +985,8 @@ public class DataNode extends Configured
   
   /**
    * get BP registration by blockPool id
-   * @param bpid
    * @return BP registration object
-   * @throws IOException
+   * @throws IOException on error
    */
   @VisibleForTesting
   public DatanodeRegistration getDNRegistrationForBP(String bpid) 
@@ -1694,8 +1690,9 @@ public class DataNode extends Configured
   /**
    * After a block becomes finalized, a datanode increases metric counter,
    * notifies namenode, and adds it to the block scanner
-   * @param block
-   * @param delHint
+   * @param block block to close
+   * @param delHint hint on which excess block to delete
+   * @param storageUuid UUID of the storage where block is stored
    */
   void closeBlock(ExtendedBlock block, String delHint, String storageUuid) {
     metrics.incrBlocksWritten();
@@ -2325,8 +2322,8 @@ public class DataNode extends Configured
    *          The corresponding replica must be an RBW or a Finalized.
    *          Its GS and numBytes will be set to
    *          the stored GS and the visible length. 
-   * @param targets
-   * @param client
+   * @param targets targets to transfer the block to
+   * @param client client name
    */
   void transferReplicaForPipelineRecovery(final ExtendedBlock b,
       final DatanodeInfo[] targets, final String client) throws IOException {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -1077,7 +1077,7 @@ class DataXceiver extends Receiver implements Runnable {
   /**
    * Utility function for sending a response.
    * 
-   * @param opStatus status message to write
+   * @param status status message to write
    * @param message message to send to the client or other DN
    */
   private void sendResponse(Status status,

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -108,8 +108,7 @@ public class DirectoryScanner implements Runnable {
     ScanInfoPerBlockPool(int sz) {super(sz);}
     
     /**
-     * Merges "that" ScanInfoPerBlockPool into this one
-     * @param that
+     * Merges {@code that} ScanInfoPerBlockPool into this one
      */
     public void addAll(ScanInfoPerBlockPool that) {
       if (that == null) return;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FinalizedReplica.java

@@ -54,7 +54,7 @@ public class FinalizedReplica extends ReplicaInfo {
 
   /**
    * Copy constructor.
-   * @param from
+   * @param from where to copy construct from
    */
   public FinalizedReplica(FinalizedReplica from) {
     super(from);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaBeingWritten.java

@@ -68,7 +68,7 @@ public class ReplicaBeingWritten extends ReplicaInPipeline {
 
   /**
    * Copy constructor.
-   * @param from
+   * @param from where to copy from
    */
   public ReplicaBeingWritten(ReplicaBeingWritten from) {
     super(from);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java

@@ -89,7 +89,7 @@ public class ReplicaInPipeline extends ReplicaInfo
 
   /**
    * Copy constructor.
-   * @param from
+   * @param from where to copy from
    */
   public ReplicaInPipeline(ReplicaInPipeline from) {
     super(from);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipelineInterface.java

@@ -40,7 +40,7 @@ public interface ReplicaInPipelineInterface extends Replica {
   
   /**
    * Set the number bytes that have acked
-   * @param bytesAcked
+   * @param bytesAcked number bytes acked
    */
   void setBytesAcked(long bytesAcked);
   

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInfo.java

@@ -100,7 +100,7 @@ abstract public class ReplicaInfo extends Block implements Replica {
 
   /**
    * Copy constructor.
-   * @param from
+   * @param from where to copy from
    */
   ReplicaInfo(ReplicaInfo from) {
     this(from, from.getVolume(), from.getDir());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaUnderRecovery.java

@@ -50,7 +50,7 @@ public class ReplicaUnderRecovery extends ReplicaInfo {
 
   /**
    * Copy constructor.
-   * @param from
+   * @param from where to copy from
    */
   public ReplicaUnderRecovery(ReplicaUnderRecovery from) {
     super(from);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaWaitingToBeRecovered.java

@@ -60,7 +60,7 @@ public class ReplicaWaitingToBeRecovered extends ReplicaInfo {
   
   /**
    * Copy constructor.
-   * @param from
+   * @param from where to copy from
    */
   public ReplicaWaitingToBeRecovered(ReplicaWaitingToBeRecovered from) {
     super(from);

+ 2 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/AvailableSpaceVolumeChoosingPolicy.java

@@ -165,13 +165,8 @@ public class AvailableSpaceVolumeChoosingPolicy<V extends FsVolumeSpi>
     }
     
     /**
-     * Check if the available space on all the volumes is roughly equal.
-     * 
-     * @param volumes the volumes to check
-     * @return true if all volumes' free space is within the configured threshold,
-     *         false otherwise.
-     * @throws IOException
-     *           in the event of error checking amount of available space
+     * @return true if all volumes' free space is within the
+     *         configured threshold, false otherwise.
      */
     public boolean areAllVolumesWithinFreeSpaceThreshold() {
       long leastAvailable = Long.MAX_VALUE;

+ 4 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java

@@ -124,16 +124,14 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
 
   /**
    * Returns the specified block's on-disk length (excluding metadata)
-   * @param b
    * @return   the specified block's on-disk length (excluding metadta)
-   * @throws IOException
+   * @throws IOException on error
    */
   public long getLength(ExtendedBlock b) throws IOException;
 
   /**
    * Get reference to the replica meta info in the replicasMap. 
    * To be called from methods that are synchronized on {@link FSDataset}
-   * @param blockId
    * @return replica from the replicas map
    */
   @Deprecated
@@ -151,8 +149,8 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   
   /**
    * Returns an input stream at specified offset of the specified block
-   * @param b
-   * @param seekOffset
+   * @param b block
+   * @param seekOffset offset with in the block to seek to
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
@@ -163,9 +161,6 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /**
    * Returns an input stream at specified offset of the specified block
    * The block is still in the tmp directory and is not finalized
-   * @param b
-   * @param blkoff
-   * @param ckoff
    * @return an input stream to read the contents of the specified block,
    *  starting at the offset
    * @throws IOException
@@ -256,7 +251,6 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Finalizes the block previously opened for writing using writeToBlock.
    * The block size is what is in the parameter b and it must match the amount
    *  of data written
-   * @param b
    * @throws IOException
    */
   public void finalizeBlock(ExtendedBlock b) throws IOException;
@@ -264,7 +258,6 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /**
    * Unfinalizes the block previously opened for writing using writeToBlock.
    * The temporary file associated with this block is deleted.
-   * @param b
    * @throws IOException
    */
   public void unfinalizeBlock(ExtendedBlock b) throws IOException;
@@ -289,14 +282,12 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
 
   /**
    * Is the block valid?
-   * @param b
    * @return - true if the specified block is valid
    */
   public boolean isValidBlock(ExtendedBlock b);
 
   /**
    * Is the block a valid RBW?
-   * @param b
    * @return - true if the specified block is a valid RBW
    */
   public boolean isValidRbw(ExtendedBlock b);
@@ -327,7 +318,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
    * Determine if the specified block is cached.
    * @param bpid Block pool id
    * @param blockIds - block id
-   * @returns true if the block is cached
+   * @return true if the block is cached
    */
   public boolean isCached(String bpid, long blockId);
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java

@@ -74,7 +74,7 @@ class BlockPoolSlice {
    * @param bpid Block pool Id
    * @param volume {@link FsVolumeImpl} to which this BlockPool belongs to
    * @param bpDir directory corresponding to the BlockPool
-   * @param conf
+   * @param conf configuration
    * @throws IOException
    */
   BlockPoolSlice(String bpid, FsVolumeImpl volume, File bpDir,

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -120,10 +120,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
 
 
   /**
-   * Returns a clone of a replica stored in data-node memory.
-   * Should be primarily used for testing.
-   * @param blockId
-   * @return
+   * This should be primarily used for testing.
+   * @return clone of replica store in datanode memory
    */
   ReplicaInfo fetchReplicaInfo(String bpid, long blockId) {
     ReplicaInfo r = volumeMap.get(bpid, blockId);
@@ -1581,7 +1579,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
         datanode.getDnConf().getXceiverStopTimeout());
   }
 
-  /** static version of {@link #initReplicaRecovery(Block, long)}. */
+  /** static version of {@link #initReplicaRecovery(RecoveringBlock)}. */
   static ReplicaRecoveryInfo initReplicaRecovery(String bpid, ReplicaMap map,
       Block block, long recoveryId, long xceiverStopTimeout) throws IOException {
     final ReplicaInfo replica = map.get(bpid, block.getBlockId());

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/MappableBlock.java

@@ -100,7 +100,6 @@ public class MappableBlock implements Closeable {
 
   /**
    * Verifies the block's checksum. This is an I/O intensive operation.
-   * @return if the block was successfully checksummed.
    */
   private static void verifyChecksum(long length,
       FileInputStream metaIn, FileChannel blockChannel, String blockFileName)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -346,7 +346,7 @@ public class BackupNode extends NameNode {
 
   /**
    * Register this backup node with the active name-node.
-   * @param nsInfo
+   * @param nsInfo namespace information
    * @throws IOException
    */
   private void registerWith(NamespaceInfo nsInfo) throws IOException {

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

@@ -216,7 +216,7 @@ public final class CacheManager {
 
   /**
    * Resets all tracked directives and pools. Called during 2NN checkpointing to
-   * reset FSNamesystem state. See {FSNamesystem{@link #clear()}.
+   * reset FSNamesystem state. See {@link FSNamesystem#clear()}.
    */
   void clear() {
     directivesById.clear();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CachedBlock.java

@@ -125,7 +125,7 @@ public final class CachedBlock implements Element,
    * @param type      If null, this parameter is ignored.
    *                  If it is non-null, we match only datanodes which
    *                  have it on this list.
-   *                  See {@link DatanodeDescriptor#CachedBlocksList#Type}
+   *                  See {@link DatanodeDescriptor.CachedBlocksList.Type}
    *                  for a description of all the lists.
    *                  
    * @return          The list of datanodes.  Modifying this list does not

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ClusterJspHelper.java

@@ -388,7 +388,6 @@ class ClusterJspHelper {
      *          is an inner map whose key is namenode, value is datanode status.
      *          reported by each namenode.
      * @param namenodeHost host name of the namenode
-     * @param decomnode update DecommissionNode with alive node status
      * @param json JSON string contains datanode status
      * @throws IOException
      */
@@ -426,7 +425,6 @@ class ClusterJspHelper {
      * @param statusMap map with key being datanode, value being an
      *          inner map (key:namenode, value:decommisionning state).
      * @param host datanode hostname
-     * @param decomnode DecommissionNode
      * @param json String
      * @throws IOException
      */
@@ -468,7 +466,6 @@ class ClusterJspHelper {
      * @param dataNodeStatusMap map with key being datanode, value being an
      *          inner map (key:namenode, value:decommisionning state).
      * @param host datanode
-     * @param decomnode DecommissionNode
      * @param json String
      */
     private static void getDecommissionNodeStatus(

+ 0 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -94,10 +94,6 @@ public class EditLogFileInputStream extends EditLogInputStream {
    * @param name filename to open
    * @param firstTxId first transaction found in file
    * @param lastTxId last transaction id found in file
-   * @throws LogHeaderCorruptException if the header is either missing or
-   *         appears to be corrupt/truncated
-   * @throws IOException if an actual IO error occurs while reading the
-   *         header
    */
   public EditLogFileInputStream(File name, long firstTxId, long lastTxId,
       boolean isInProgress) {

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java

@@ -127,14 +127,14 @@ public abstract class EditLogOutputStream implements Closeable {
   }
   
   /**
-   * Return total time spent in {@link #flushAndSync()}
+   * Return total time spent in {@link #flushAndSync(boolean)}
    */
   long getTotalSyncTime() {
     return totalTimeSync;
   }
 
   /**
-   * Return number of calls to {@link #flushAndSync()}
+   * Return number of calls to {@link #flushAndSync(boolean)}
    */
   protected long getNumSync() {
     return numSync;

+ 8 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -524,7 +524,7 @@ public class FSDirectory implements Closeable {
   /**
    * @throws SnapshotAccessControlException 
    * @see #unprotectedRenameTo(String, String, long)
-   * @deprecated Use {@link #renameTo(String, String, Rename...)} instead.
+   * @deprecated Use {@link #renameTo(String, String, boolean, Rename...)}
    */
   @Deprecated
   boolean renameTo(String src, String dst, boolean logRetryCache) 
@@ -581,7 +581,7 @@ public class FSDirectory implements Closeable {
    * @throws QuotaExceededException if the operation violates any quota limit
    * @throws FileAlreadyExistsException if the src is a symlink that points to dst
    * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @deprecated See {@link #renameTo(String, String)}
+   * @deprecated See {@link #renameTo(String, String, boolean, Rename...)}
    */
   @Deprecated
   boolean unprotectedRenameTo(String src, String dst, long timestamp)
@@ -1844,7 +1844,7 @@ public class FSDirectory implements Closeable {
   
   /** 
    * update quota of each inode and check to see if quota is exceeded. 
-   * See {@link #updateCount(INode[], int, long, long, boolean)}
+   * See {@link #updateCount(INodesInPath, long, long, boolean)}
    */ 
   private void updateCountNoQuotaCheck(INodesInPath inodesInPath,
       int numOfINodes, long nsDelta, long dsDelta) {
@@ -1928,14 +1928,13 @@ public class FSDirectory implements Closeable {
 
    * @param src string representation of the path to the directory
    * @param permissions the permission of the directory
-   * @param isAutocreate if the permission of the directory should inherit
+   * @param inheritPermission if the permission of the directory should inherit
    *                          from its parent or not. u+wx is implicitly added to
    *                          the automatically created directories, and to the
    *                          given directory if inheritPermission is true
    * @param now creation time
    * @return true if the operation succeeds false otherwise
-   * @throws FileNotFoundException if an ancestor or itself is a file
-   * @throws QuotaExceededException if directory creation violates 
+   * @throws QuotaExceededException if directory creation violates
    *                                any quota limit
    * @throws UnresolvedLinkException if a symlink is encountered in src.                      
    * @throws SnapshotAccessControlException if path is in RO snapshot
@@ -2064,7 +2063,7 @@ public class FSDirectory implements Closeable {
   /**
    * Add the given child to the namespace.
    * @param src The full path name of the child node.
-   * @throw QuotaExceededException is thrown if it violates quota limit
+   * @throws QuotaExceededException is thrown if it violates quota limit
    */
   private boolean addINode(String src, INode child
       ) throws QuotaExceededException, UnresolvedLinkException {
@@ -2260,7 +2259,7 @@ public class FSDirectory implements Closeable {
    * Its ancestors are stored at [0, pos-1].
    * @return false if the child with this name already exists; 
    *         otherwise return true;
-   * @throw QuotaExceededException is thrown if it violates quota limit
+   * @throws QuotaExceededException is thrown if it violates quota limit
    */
   private boolean addChild(INodesInPath iip, int pos,
       INode child, boolean checkQuota) throws QuotaExceededException {
@@ -2447,7 +2446,7 @@ public class FSDirectory implements Closeable {
   /**
    * See {@link ClientProtocol#setQuota(String, long, long)} for the contract.
    * Sets quota for for a directory.
-   * @returns INodeDirectory if any of the quotas have changed. null other wise.
+   * @return INodeDirectory if any of the quotas have changed. null other wise.
    * @throws FileNotFoundException if the path does not exist.
    * @throws PathIsNotDirectoryException if the path is not a directory.
    * @throws QuotaExceededException if the directory tree size is 

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

@@ -424,7 +424,6 @@ public class FSEditLog implements LogsPurgeable {
 
   /**
    * Wait if an automatic sync is scheduled
-   * @throws InterruptedException
    */
   synchronized void waitIfAutoSyncScheduled() {
     try {
@@ -802,7 +801,8 @@ public class FSEditLog implements LogsPurgeable {
   /** Add set namespace quota record to edit log
    * 
    * @param src the string representation of the path to a directory
-   * @param quota the directory size limit
+   * @param nsQuota namespace quota
+   * @param dsQuota diskspace quota
    */
   void logSetQuota(String src, long nsQuota, long dsQuota) {
     SetQuotaOp op = SetQuotaOp.getInstance(cache.get())
@@ -1408,8 +1408,9 @@ public class FSEditLog implements LogsPurgeable {
    * Select a list of input streams.
    * 
    * @param fromTxId first transaction in the selected streams
-   * @param toAtLeast the selected streams must contain this transaction
-   * @param inProgessOk set to true if in-progress streams are OK
+   * @param toAtLeastTxId the selected streams must contain this transaction
+   * @param recovery recovery context
+   * @param inProgressOk set to true if in-progress streams are OK
    */
   public synchronized Collection<EditLogInputStream> selectInputStreams(
       long fromTxId, long toAtLeastTxId, MetaRecoveryContext recovery,

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -996,9 +996,6 @@ public class FSEditLogLoader {
    * If there are invalid or corrupt transactions in the middle of the stream,
    * validateEditLog will skip over them.
    * This reads through the stream but does not close it.
-   *
-   * @throws IOException if the stream cannot be read due to an IO error (eg
-   *                     if the log does not exist)
    */
   static EditLogValidation validateEditLog(EditLogInputStream in) {
     long lastPos = 0;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -668,8 +668,8 @@ public abstract class FSEditLogOp {
   }
 
   /**
-   * {@literal @AtMostOnce} for {@link ClientProtocol#startFile} and
-   * {@link ClientProtocol#appendFile}
+   * {@literal @AtMostOnce} for {@link ClientProtocol#create} and
+   * {@link ClientProtocol#append}
    */
   static class AddOp extends AddCloseOp {
     private AddOp() {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -156,7 +156,7 @@ public class FSImage implements Closeable {
    * directory to allow them to format anyway. Otherwise, returns
    * false, unless 'force' is specified.
    * 
-   * @param force format regardless of whether dirs exist
+   * @param force if true, format regardless of whether dirs exist
    * @param interactive prompt the user when a dir exists
    * @return true if formatting should proceed
    * @throws IOException if some storage cannot be accessed
@@ -1002,7 +1002,6 @@ public class FSImage implements Closeable {
   /**
    * Save the contents of the FS image to a new image file in each of the
    * current storage directories.
-   * @param canceler
    */
   public synchronized void saveNamespace(FSNamesystem source, NameNodeFile nnf,
       Canceler canceler) throws IOException {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -485,7 +485,7 @@ public class FSImageFormat {
    /**
     * Load all children of a directory
     * 
-    * @param in
+    * @param in input to load from
     * @param counter Counter to increment for namenode startup progress
     * @return number of child inodes read
     * @throws IOException

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -206,7 +206,7 @@ public class FSImageSerialization {
   /**
    * Reading the path from the image and converting it to byte[][] directly
    * this saves us an array copy and conversions to and from String
-   * @param in
+   * @param in input to read from
    * @return the array each element of which is a byte[] representation 
    *            of a path component
    * @throws IOException

+ 17 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -1339,7 +1339,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Returns edit directories that are shared between primary and secondary.
    * @param conf configuration
-   * @return Collection of edit directories.
+   * @return collection of edit directories from {@code conf}
    */
   public static List<URI> getSharedEditsDirs(Configuration conf) {
     // don't use getStorageDirs here, because we want an empty default
@@ -1774,9 +1774,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * before we start actual move.
    * 
    * This does not support ".inodes" relative path
-   * @param target target file path to concatenate into
-   * @param srcs files that are concatenated
-   * @throws IOException
+   * @param target target to concat into
+   * @param srcs file that will be concatenated
+   * @throws IOException on error
    */
   void concat(String target, String [] srcs) 
       throws IOException, UnresolvedLinkException {
@@ -4075,11 +4075,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   /**
-   *
-   * @param pendingFile
-   * @param storedBlock
+   * @param pendingFile open file that needs to be closed
+   * @param storedBlock last block
    * @return Path of the file that was closed.
-   * @throws IOException
+   * @throws IOException on error
    */
   @VisibleForTesting
   String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
@@ -4287,7 +4286,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * Perform resource checks and cache the results.
-   * @throws IOException
    */
   void checkAvailableResources() {
     Preconditions.checkState(nnResourceChecker != null,
@@ -5338,7 +5336,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * Leave safe mode.
-   * @throws IOException
    */
   void leaveSafeMode() {
     writeLock();
@@ -5755,7 +5752,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Sets the generation stamp that delineates random and sequentially
    * allocated block IDs.
-   * @param stamp
+   * @param stamp set generation stamp limit to this value
    */
   void setGenerationStampV1Limit(long stamp) {
     Preconditions.checkState(generationStampV1Limit ==
@@ -5840,7 +5837,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Determine whether the block ID was randomly generated (legacy) or
    * sequentially generated. The generation stamp value is used to
    * make the distinction.
-   * @param block
    * @return true if the block ID was randomly generated, false otherwise.
    */
   boolean isLegacyBlock(Block block) {
@@ -6077,7 +6073,6 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Release (unregister) backup node.
    * <p>
    * Find and remove the backup stream corresponding to the node.
-   * @param registration
    * @throws IOException
    */
   void releaseBackupNode(NamenodeRegistration registration)
@@ -6213,8 +6208,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * @param renewer Renewer information
-   * @return Token<DelegationTokenIdentifier>
-   * @throws IOException
+   * @return delegation toek
+   * @throws IOException on error
    */
   Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
@@ -6255,10 +6250,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * 
-   * @param token delegation token
-   * @return New expiryTime of the token
-   * @throws InvalidToken
-   * @throws IOException
+   * @param token token to renew
+   * @return new expiryTime of the token
+   * @throws InvalidToken if {@code token} is invalid
+   * @throws IOException on other errors
    */
   long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
@@ -6289,8 +6284,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * 
-   * @param token delegation token that needs to be canceled
-   * @throws IOException
+   * @param token token to cancel
+   * @throws IOException on error
    */
   void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
@@ -7201,7 +7196,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /**
    * Update internal state to indicate that a rolling upgrade is in progress.
-   * @param startTime start time of the rolling upgrade
+   * @param startTime rolling upgrade start time
    */
   void startRollingUpgradeInternal(long startTime)
       throws IOException {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java

@@ -168,7 +168,7 @@ public class FileJournalManager implements JournalManager {
 
   /**
    * Find all editlog segments starting at or above the given txid.
-   * @param fromTxId the txnid which to start looking
+   * @param firstTxId the txnid which to start looking
    * @param inProgressOk whether or not to include the in-progress edit log 
    *        segment       
    * @return a list of remote edit logs

+ 3 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -652,9 +652,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
 
 
   /**
-   * Breaks file path into components.
-   * @param path
-   * @return array of byte arrays each of which represents 
+   * Breaks {@code path} into components.
+   * @return array of byte arrays each of which represents
    * a single path component.
    */
   static byte[][] getPathComponents(String path) {
@@ -673,8 +672,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
   }
 
   /**
-   * Splits an absolute path into an array of path components.
-   * @param path
+   * Splits an absolute {@code path} into an array of path components.
    * @throws AssertionError if the given path is invalid.
    * @return array of path components.
    */

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -402,7 +402,6 @@ public class LeaseManager {
   /**
    * Get the list of inodes corresponding to valid leases.
    * @return list of inodes
-   * @throws UnresolvedLinkException
    */
   Map<String, INodeFile> getINodesUnderConstruction() {
     Map<String, INodeFile> inodes = new TreeMap<String, INodeFile>();

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LogsPurgeable.java

@@ -42,7 +42,6 @@ interface LogsPurgeable {
    * 
    * @param fromTxId the first transaction id we want to read
    * @param inProgressOk whether or not in-progress streams should be returned
-   * @return a list of streams
    * @throws IOException if the underlying storage has an error or is otherwise
    * inaccessible
    */

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -425,8 +425,7 @@ public class NNStorage extends Storage implements Closeable,
   
   /**
    * Write last checkpoint time into a separate file.
-   *
-   * @param sd
+   * @param sd storage directory
    * @throws IOException
    */
   void writeTransactionIdFile(StorageDirectory sd, long txid) throws IOException {

+ 2 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -355,8 +355,6 @@ public class NameNode implements NameNodeStatusMXBean {
 
 
   /**
-   * TODO:FEDERATION
-   * @param filesystemURI
    * @return address of file system
    */
   public static InetSocketAddress getAddress(URI filesystemURI) {
@@ -799,8 +797,8 @@ public class NameNode implements NameNodeStatusMXBean {
    * Interactively confirm that formatting is desired 
    * for each existing directory and format them.
    * 
-   * @param conf
-   * @param force
+   * @param conf configuration to use
+   * @param force if true, format regardless of whether dirs exist
    * @return true if formatting was aborted, false otherwise
    * @throws IOException
    */

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -1176,9 +1176,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   /**
    * Verify version.
-   * 
-   * @param version
-   * @throws IOException
+   * @param version layout version
+   * @throws IOException on layout version mismatch
    */
   void verifyLayoutVersion(int version) throws IOException {
     if (version != HdfsConstants.NAMENODE_LAYOUT_VERSION)

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -159,7 +159,6 @@ public class NamenodeFsck {
    * @param totalDatanodes number of live datanodes
    * @param minReplication minimum replication
    * @param remoteAddress source address of the fsck request
-   * @throws IOException
    */
   NamenodeFsck(Configuration conf, NameNode namenode,
       NetworkTopology networktopology, 

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -209,7 +209,6 @@ public class SecondaryNameNode implements Runnable {
   
   /**
    * Initialize SecondaryNameNode.
-   * @param commandLineOpts
    */
   private void initialize(final Configuration conf,
       CommandLineOpts commandLineOpts) throws IOException {

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

@@ -130,8 +130,8 @@ abstract public class HAState {
    * Check if an operation is supported in a given state.
    * @param context HA context
    * @param op Type of the operation.
-   * @throws UnsupportedActionException if a given type of operation is not
-   *           supported in this state.
+   * @throws StandbyException if a given type of operation is not
+   *           supported in standby state
    */
   public abstract void checkOperation(final HAContext context, final OperationCategory op)
       throws StandbyException;

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiff.java

@@ -86,7 +86,6 @@ abstract class AbstractINodeDiff<N extends INode,
     return posteriorDiff;
   }
 
-  /** @return the posterior diff. */
   final void setPosterior(D posterior) {
     posteriorDiff = posterior;
   }

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockIdCommand.java

@@ -32,7 +32,6 @@ public class BlockIdCommand extends DatanodeCommand {
 
   /**
    * Create BlockCommand for the given action
-   * @param blocks blocks related to the action
    */
   public BlockIdCommand(int action, String poolId, long[] blockIds) {
     super(action);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java

@@ -119,9 +119,9 @@ public interface DatanodeProtocol {
    * and should be deleted.  This function is meant to upload *all*
    * the locally-stored blocks.  It's invoked upon startup and then
    * infrequently afterwards.
-   * @param registration
-   * @param poolId - the block pool ID for the blocks
-   * @param reports - report of blocks per storage
+   * @param registration datanode registration
+   * @param poolId the block pool ID for the blocks
+   * @param reports report of blocks per storage
    *     Each finalized block is represented as 3 longs. Each under-
    *     construction replica is represented as 4 longs.
    *     This is done instead of Block[] to reduce memory used by block reports.

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeStorage.java

@@ -48,8 +48,6 @@ public class DatanodeStorage {
 
   /**
    * Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
-   *
-   * @param storageID
    */
   public DatanodeStorage(String storageID) {
     this(storageID, State.NORMAL, StorageType.DEFAULT);

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

@@ -39,7 +39,7 @@ public abstract class ServerCommand {
    * 
    * @see DatanodeProtocol
    * @see NamenodeProtocol
-   * @param action
+   * @param action protocol specific action
    */
   public ServerCommand(int action) {
     this.action = action;

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java

@@ -130,9 +130,6 @@ public class DFSck extends Configured implements Tool {
     out.println(USAGE + "\n");
     ToolRunner.printGenericCommandUsage(out);
   }
-  /**
-   * @param args
-   */
   @Override
   public int run(final String[] args) throws IOException {
     if (args.length == 0) {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/GetConf.java

@@ -167,8 +167,7 @@ public class GetConf extends Configured implements Tool {
     }
 
     
-    /** Method to be overridden by sub classes for specific behavior 
-     * @param args */
+    /** Method to be overridden by sub classes for specific behavior */
     int doWorkInternal(GetConf tool, String[] args) throws Exception {
 
       String value = tool.getConf().getTrimmed(key);

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/HDFSConcat.java

@@ -30,9 +30,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 @InterfaceAudience.Private
 public class HDFSConcat {
   private final static String def_uri = "hdfs://localhost:9000";
-  /**
-   * @param args
-   */
+
   public static void main(String... args) throws IOException {
 
     if(args.length < 2) {

+ 1 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/JMXGet.java

@@ -217,7 +217,7 @@ public class JMXGet {
   }
 
   /**
-   * @param msg
+   * @param msg error message
    */
   private static void err(String msg) {
     System.err.println(msg);
@@ -274,13 +274,7 @@ public class JMXGet {
     return commandLine;
   }
 
-  /**
-   * main
-   * 
-   * @param args
-   */
   public static void main(String[] args) {
-
     int res = -1;
 
     // parse arguments

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/BinaryEditsVisitor.java

@@ -37,8 +37,7 @@ public class BinaryEditsVisitor implements OfflineEditsVisitor {
 
   /**
    * Create a processor that writes to a given file
-   *
-   * @param filename Name of file to write output to
+   * @param outputName Name of file to write output to
    */
   public BinaryEditsVisitor(String outputName) throws IOException {
     this.elfos = new EditLogFileOutputStream(new Configuration(),

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/DataTransferThrottler.java

@@ -63,8 +63,6 @@ public class DataTransferThrottler {
   /**
    * Sets throttle bandwidth. This takes affect latest by the end of current
    * period.
-   * 
-   * @param bytesPerSecond 
    */
   public synchronized void setBandwidth(long bytesPerSecond) {
     if ( bytesPerSecond <= 0 ) {

+ 1 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/AclPermissionParam.java

@@ -60,10 +60,7 @@ public class AclPermissionParam extends StringParam {
   }
 
   /**
-   * Parse the list of AclEntry and returns aclspec.
-   * 
-   * @param List <AclEntry>
-   * @return String
+   * @return parse {@code aclEntry} and return aclspec
    */
   private static String parseAclSpec(List<AclEntry> aclEntry) {
     return StringUtils.join(aclEntry, ",");

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BenchmarkThroughput.java

@@ -229,7 +229,7 @@ public class BenchmarkThroughput extends Configured implements Tool {
   }
 
   /**
-   * @param args
+   * @param args arguments
    */
   public static void main(String[] args) throws Exception {
     int res = ToolRunner.run(new HdfsConfiguration(),

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -1386,8 +1386,8 @@ public class MiniDFSCluster {
   /**
    * Finalize cluster for the namenode at the given index 
    * @see MiniDFSCluster#finalizeCluster(Configuration)
-   * @param nnIndex
-   * @param conf
+   * @param nnIndex index of the namenode
+   * @param conf configuration
    * @throws Exception
    */
   public void finalizeCluster(int nnIndex, Configuration conf) throws Exception {
@@ -2248,7 +2248,7 @@ public class MiniDFSCluster {
    * to determine the location of the storage of a DN instance in the mini cluster
    * @param dnIndex datanode index
    * @param dirIndex directory index.
-   * @return
+   * @return storage directory path
    */
   private static String getStorageDirPath(int dnIndex, int dirIndex) {
     return "data/data" + (2 * dnIndex + 1 + dirIndex);
@@ -2274,8 +2274,8 @@ public class MiniDFSCluster {
   }
   /**
    * Get directory relative to block pool directory in the datanode
-   * @param storageDir
-   * @return current directory
+   * @param storageDir storage directory
+   * @return current directory in the given storage directory
    */
   public static String getBPDir(File storageDir, String bpid, String dirName) {
     return getBPDir(storageDir, bpid) + dirName + "/";

+ 2 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java

@@ -101,7 +101,6 @@ public class BlockManagerTestUtil {
   }
 
   /**
-   * @param blockManager
    * @return replication monitor thread instance from block manager.
    */
   public static Daemon getReplicationThread(final BlockManager blockManager)
@@ -111,7 +110,6 @@ public class BlockManagerTestUtil {
   
   /**
    * Stop the replication monitor thread
-   * @param blockManager
    */
   public static void stopReplicationThread(final BlockManager blockManager) 
       throws IOException {
@@ -126,7 +124,6 @@ public class BlockManagerTestUtil {
   }
 
   /**
-   * @param blockManager
    * @return corruptReplicas from block manager
    */
   public static  CorruptReplicasMap getCorruptReplicas(final BlockManager blockManager){
@@ -135,7 +132,6 @@ public class BlockManagerTestUtil {
   }
 
   /**
-   * @param blockManager
    * @return computed block replication and block invalidation work that can be
    *         scheduled on data-nodes.
    * @throws IOException
@@ -158,7 +154,7 @@ public class BlockManagerTestUtil {
    * regardless of invalidation/replication limit configurations.
    * 
    * NB: you may want to set
-   * {@link DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY} to
+   * {@link DFSConfigKeys#DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY} to
    * a high value to ensure that all work is calculated.
    */
   public static int computeAllPendingWork(BlockManager bm) {
@@ -200,7 +196,7 @@ public class BlockManagerTestUtil {
   /**
    * Change whether the block placement policy will prefer the writer's
    * local Datanode or not.
-   * @param prefer
+   * @param prefer if true, prefer local node
    */
   public static void setWritingPrefersLocalNode(
       BlockManager bm, boolean prefer) {

+ 0 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java

@@ -171,9 +171,6 @@ public abstract class BlockReportTestBase {
    * Utility routine to send block reports to the NN, either in a single call
    * or reporting one storage per call.
    *
-   * @param dnR
-   * @param poolId
-   * @param reports
    * @throws IOException
    */
   protected abstract void sendBlockReports(DatanodeRegistration dnR, String poolId,

+ 3 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

@@ -833,8 +833,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     
     /**
      * An input stream of size l with repeated bytes
-     * @param l
-     * @param iRepeatedData
+     * @param l size of the stream
+     * @param iRepeatedData byte that is repeated in the stream
      */
     SimulatedInputStream(long l, byte iRepeatedData) {
       length = l;
@@ -843,17 +843,14 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
     
     /**
      * An input stream of of the supplied data
-     * 
-     * @param iData
+     * @param iData data to construct the stream
      */
     SimulatedInputStream(byte[] iData) {
       data = iData;
       length = data.length;
-      
     }
     
     /**
-     * 
      * @return the lenght of the input stream
      */
     long getLength() {

+ 2 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -131,14 +131,10 @@ public class CreateEditsLog {
     printUsageExit();
   }
   /**
-   * @param args
+   * @param args arguments
    * @throws IOException 
    */
-  public static void main(String[] args) 
-      throws IOException {
-
-
-
+  public static void main(String[] args)  throws IOException {
     long startingBlockId = 1;
     int numFiles = 0;
     short replication = 1;

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java

@@ -202,7 +202,7 @@ public class NNThroughputBenchmark implements Tool {
      * {@link #executeOp(int, int, String)}, which can have different meanings
      * depending on the operation performed.
      * 
-     * @param daemonId
+     * @param daemonId id of the daemon calling this method
      * @return the argument
      */
     abstract String getExecutionArgument(int daemonId);
@@ -322,11 +322,10 @@ public class NNThroughputBenchmark implements Tool {
     /**
      * Parse first 2 arguments, corresponding to the "-op" option.
      * 
-     * @param args
+     * @param args argument list
      * @return true if operation is all, which means that options not related
      * to this operation should be ignored, or false otherwise, meaning
      * that usage should be printed when an unrelated option is encountered.
-     * @throws IOException
      */
     protected boolean verifyOpArgument(List<String> args) {
       if(args.size() < 2 || ! args.get(0).startsWith("-op"))