Explorar el Código

HDFS-13942. [JDK10] Fix javadoc errors in hadoop-hdfs module. Contributed by Dinesh Chitlangia.

Akira Ajisaka hace 6 años
padre
commit
fac9f91b29
Se han modificado 65 ficheros con 310 adiciones y 246 borrados
  1. 8 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
  5. 8 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  6. 15 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  7. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java
  8. 2 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CombinedHostFileManager.java
  9. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java
  10. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java
  11. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java
  12. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostSet.java
  13. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java
  14. 36 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  15. 10 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockRecoveryWorker.java
  16. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockScanner.java
  17. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
  18. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  19. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java
  20. 0 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/FileIoProvider.java
  21. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/VolumeScanner.java
  22. 4 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java
  23. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsDatasetSpi.java
  24. 7 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/FsVolumeSpi.java
  25. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/OutlierDetector.java
  26. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java
  27. 5 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java
  28. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java
  29. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java
  30. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclStorage.java
  31. 28 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
  32. 3 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  33. 19 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  34. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  35. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeReference.java
  36. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
  37. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalManager.java
  38. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
  39. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/MetaRecoveryContext.java
  40. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  41. 4 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  42. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Quota.java
  43. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java
  44. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrPermissionFilter.java
  45. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/XAttrStorage.java
  46. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java
  47. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/DiffListBySkipList.java
  48. 2 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/BlockStorageMovementNeeded.java
  49. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/DatanodeCacheManager.java
  50. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java
  51. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StartupProgressView.java
  52. 13 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/metrics/TopMetrics.java
  53. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindow.java
  54. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindowManager.java
  55. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java
  56. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
  57. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamenodeProtocol.java
  58. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java
  59. 8 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
  60. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsViewer.java
  61. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsVisitor.java
  62. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/StatisticsEditsVisitor.java
  63. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/NameDistributionVisitor.java
  64. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java
  65. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java

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

@@ -160,7 +160,8 @@ public class DFSUtil {
   /**
    * Comparator for sorting DataNodeInfo[] based on
    * stale, decommissioned and entering_maintenance states.
-   * Order: live -> stale -> entering_maintenance -> decommissioned
+   * Order: live {@literal ->} stale {@literal ->} entering_maintenance
+   * {@literal ->} decommissioned
    */
   @InterfaceAudience.Private 
   public static class ServiceAndStaleComparator extends ServiceComparator {
@@ -390,7 +391,8 @@ public class DFSUtil {
    * @param conf Configuration
    * @param nsId the nameservice whose NNs addresses we want.
    * @param defaultValue default address to return in case key is not found.
-   * @return A map from nnId -> RPC address of each NN in the nameservice.
+   * @return A map from nnId {@literal ->} RPC address of each NN in the
+   * nameservice.
    */
   public static Map<String, InetSocketAddress> getRpcAddressesForNameserviceId(
       Configuration conf, String nsId, String defaultValue) {
@@ -1289,7 +1291,8 @@ public class DFSUtil {
    * @param conf configuration
    * @param protocol Protocol interface
    * @param service service that implements the protocol
-   * @param server RPC server to which the protocol & implementation is added to
+   * @param server RPC server to which the protocol &amp; implementation is
+   *               added to
    * @throws IOException
    */
   public static void addPBProtocol(Configuration conf, Class<?> protocol,
@@ -1357,7 +1360,8 @@ public class DFSUtil {
    * @param conf Configuration
    * @param nsId the nameservice whose NNs addresses we want.
    * @param defaultValue default address to return in case key is not found.
-   * @return A map from nnId -> Web address of each NN in the nameservice.
+   * @return A map from nnId {@literal ->} Web address of each NN in the
+   * nameservice.
    */
   public static Map<String, InetSocketAddress> getWebAddressesForNameserviceId(
       Configuration conf, String nsId, String defaultValue) {

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

@@ -201,7 +201,7 @@ public abstract class BlockListAsLongs implements Iterable<BlockReportReplica> {
   /**
    * Very efficient encoding of the block report into a ByteString to avoid
    * the overhead of protobuf repeating fields.  Primitive repeating fields
-   * require re-allocs of an ArrayList<Long> and the associated (un)boxing
+   * require re-allocs of an ArrayList&lt;Long&gt; and the associated (un)boxing
    * overhead which puts pressure on GC.
    * 
    * The structure of the buffer is as follows:

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java

@@ -168,7 +168,7 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
     return VOID_JOURNAL_RESPONSE;
   }
 
-  /** @see JournalProtocol#heartbeat */
+  /** @see QJournalProtocol#heartbeat */
   @Override
   public HeartbeatResponseProto heartbeat(RpcController controller,
       HeartbeatRequestProto req) throws ServiceException {

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

@@ -220,7 +220,7 @@ public class BlockTokenSecretManager extends
   }
 
   /**
-   * Update block keys if update time > update interval.
+   * Update block keys if update time {@literal >} update interval.
    * @return true if the keys are updated.
    */
   public synchronized boolean updateKeys(final long updateTime) throws IOException {

+ 8 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -78,7 +78,7 @@ import com.google.common.base.Preconditions;
  * <p>SYNOPSIS
  * <pre>
  * To start:
- *      bin/start-balancer.sh [-threshold <threshold>]
+ *      bin/start-balancer.sh [-threshold {@literal <threshold>}]
  *      Example: bin/ start-balancer.sh 
  *                     start the balancer with a default threshold of 10%
  *               bin/ start-balancer.sh -threshold 5
@@ -113,13 +113,14 @@ import com.google.common.base.Preconditions;
  * <p>A system property that limits the balancer's use of bandwidth is 
  * defined in the default configuration file:
  * <pre>
- * <property>
- *   <name>dfs.datanode.balance.bandwidthPerSec</name>
- *   <value>1048576</value>
- * <description>  Specifies the maximum bandwidth that each datanode 
+ * &lt;property&gt;
+ *   &lt;name&gt;dfs.datanode.balance.bandwidthPerSec&lt;/name&gt;
+ *   &lt;value&gt;1048576&lt;/value&gt;
+ * &lt;description&gt;  Specifies the maximum bandwidth that each datanode
  * can utilize for the balancing purpose in term of the number of bytes 
- * per second. </description>
- * </property>
+ * per second.
+ * &lt;/description&gt;
+ * &lt;/property&gt;
  * </pre>
  * 
  * <p>This property determines the maximum speed at which a block will be 

+ 15 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -143,12 +143,13 @@ import org.slf4j.LoggerFactory;
  * If any of the replica is in maintenance mode, the safety property
  * is extended as follows. These property still apply for the case of zero
  * maintenance replicas, thus we can use these safe property for all scenarios.
- * a. # of live replicas >= # of min replication for maintenance.
- * b. # of live replicas <= # of expected redundancy.
- * c. # of live replicas and maintenance replicas >= # of expected redundancy.
+ * a. # of live replicas &gt;= # of min replication for maintenance.
+ * b. # of live replicas &lt;= # of expected redundancy.
+ * c. # of live replicas and maintenance replicas &gt;= # of expected
+ * redundancy.
  *
  * For regular replication, # of min live replicas for maintenance is determined
- * by DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY. This number has to <=
+ * by DFS_NAMENODE_MAINTENANCE_REPLICATION_MIN_KEY. This number has to &lt;=
  * DFS_NAMENODE_REPLICATION_MIN_KEY.
  * For erasure encoding, # of min live replicas for maintenance is
  * BlockInfoStriped#getRealDataBlockNum.
@@ -305,7 +306,7 @@ public class BlockManager implements BlockStatsMXBean {
   private final double storageInfoDefragmentRatio;
 
   /**
-   * Mapping: Block -> { BlockCollection, datanodes, self ref }
+   * Mapping: Block {@literal ->} { BlockCollection, datanodes, self ref }
    * Updated only in response to client-sent information.
    */
   final BlocksMap blocksMap;
@@ -321,7 +322,9 @@ public class BlockManager implements BlockStatsMXBean {
   private final BlockReportProcessingThread blockReportThread =
       new BlockReportProcessingThread();
 
-  /** Store blocks -> datanodedescriptor(s) map of corrupt replicas */
+  /**
+   * Store blocks {@literal ->} datanodedescriptor(s) map of corrupt replicas.
+   */
   final CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
 
   /**
@@ -2105,7 +2108,7 @@ public class BlockManager implements BlockStatsMXBean {
    * Choose target datanodes for creating a new block.
    * 
    * @throws IOException
-   *           if the number of targets < minimum replication.
+   *           if the number of targets {@literal <} minimum replication.
    * @see BlockPlacementPolicy#chooseTarget(String, int, Node,
    *      Set, long, List, BlockStoragePolicy, EnumSet)
    */
@@ -2487,7 +2490,8 @@ public class BlockManager implements BlockStatsMXBean {
 
   /**
    * The given storage is reporting all its blocks.
-   * Update the (storage-->block list) and (block-->storage list) maps.
+   * Update the (storage{@literal -->}block list) and
+   * (block{@literal -->}storage list) maps.
    *
    * @return true if all known storages of the given DN have finished reporting.
    * @throws IOException
@@ -3777,8 +3781,8 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Modify (block-->datanode) map. Possibly generate replication tasks, if the
-   * removed block is still valid.
+   * Modify (block{@literal -->}datanode) map. Possibly generate replication
+   * tasks, if the removed block is still valid.
    */
   public void removeStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
     blockLog.debug("BLOCK* removeStoredBlock: {} from {}", storedBlock, node);
@@ -4341,7 +4345,7 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   /**
-   * Get blocks to invalidate for <i>nodeId</i>
+   * Get blocks to invalidate for {@code nodeId}
    * in {@link #invalidateBlocks}.
    *
    * @return number of blocks scheduled for removal during this iteration.

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

@@ -152,7 +152,6 @@ public abstract class BlockPlacementPolicy {
 
   /**
    * Check if the move is allowed. Used by balancer and other tools.
-   * @
    *
    * @param candidates all replicas including source and target
    * @param source source replica of the move

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

@@ -47,13 +47,11 @@ import org.apache.hadoop.hdfs.util.CombinedHostsFileReader;
 /**
  * This class manages datanode configuration using a json file.
  * Please refer to {@link CombinedHostsFileReader} for the json format.
- * <p/>
- * <p/>
+ * <p>
  * Entries may or may not specify a port.  If they don't, we consider
  * them to apply to every DataNode on that host. The code canonicalizes the
  * entries into IP addresses.
- * <p/>
- * <p/>
+ * <p>
  * The code ignores all entries that the DNS fails to resolve their IP
  * addresses. This is okay because by default the NN rejects the registrations
  * of DNs when it fails to do a forward and reverse lookup. Note that DNS

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CorruptReplicasMap.java

@@ -38,7 +38,7 @@ import com.google.common.annotations.VisibleForTesting;
  * corrupt. While reporting replicas of a Block, we hide any corrupt
  * copies. These copies are removed once Block is found to have 
  * expected number of good replicas.
- * Mapping: Block -> TreeSet<DatanodeDescriptor> 
+ * Mapping: Block {@literal -> TreeSet<DatanodeDescriptor>}
  */
 
 @InterfaceAudience.Private

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminManager.java

@@ -57,7 +57,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * Manages decommissioning and maintenance state for DataNodes. A background
  * monitor thread periodically checks the status of DataNodes that are
  * decommissioning or entering maintenance state.
- * <p/>
+ * <p>
  * A DataNode can be decommissioned in a few situations:
  * <ul>
  * <li>If a DN is dead, it is decommissioned immediately.</li>
@@ -72,11 +72,11 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * determine if they can be DECOMMISSIONED. The monitor also prunes this list
  * as blocks become replicated, so monitor scans will become more efficient
  * over time.
- * <p/>
+ * <p>
  * DECOMMISSION_INPROGRESS nodes that become dead do not progress to
  * DECOMMISSIONED until they become live again. This prevents potential
  * durability loss for singly-replicated blocks (see HDFS-6791).
- * <p/>
+ * <p>
  * DataNodes can also be put under maintenance state for any short duration
  * maintenance operations. Unlike decommissioning, blocks are not always
  * re-replicated for the DataNodes to enter maintenance state. When the
@@ -88,7 +88,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  * of maintenance expiry time. When DataNodes don't transition or join the
  * cluster back by expiry time, blocks are re-replicated just as in
  * decommissioning case as to avoid read or write performance degradation.
- * <p/>
+ * <p>
  * This class depends on the FSNamesystem lock for synchronization.
  */
 @InterfaceAudience.Private

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostFileManager.java

@@ -33,17 +33,16 @@ import java.util.HashSet;
 
 /**
  * This class manages the include and exclude files for HDFS.
- * <p/>
+ * <p>
  * These files control which DataNodes the NameNode expects to see in the
  * cluster.  Loosely speaking, the include file, if it exists and is not
  * empty, is a list of everything we expect to see.  The exclude file is
  * a list of everything we want to ignore if we do see it.
- * <p/>
+ * <p>
  * Entries may or may not specify a port.  If they don't, we consider
  * them to apply to every DataNode on that host. The code canonicalizes the
  * entries into IP addresses.
- * <p/>
- * <p/>
+ * <p>
  * The code ignores all entries that the DNS fails to resolve their IP
  * addresses. This is okay because by default the NN rejects the registrations
  * of DNs when it fails to do a forward and reverse lookup. Note that DNS

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HostSet.java

@@ -35,9 +35,9 @@ import java.util.Map;
 
 /**
  * The HostSet allows efficient queries on matching wildcard addresses.
- * <p/>
+ * <p>
  * For InetSocketAddress A and B with the same host address,
- * we define a partial order between A and B, A <= B iff A.getPort() == B
+ * we define a partial order between A and B, A &lt;= B iff A.getPort() == B
  * .getPort() || B.getPort() == 0.
  */
 public class HostSet implements Iterable<InetSocketAddress> {
@@ -46,7 +46,7 @@ public class HostSet implements Iterable<InetSocketAddress> {
 
   /**
    * The function that checks whether there exists an entry foo in the set
-   * so that foo <= addr.
+   * so that foo &lt;= addr.
    */
   boolean matchedBy(InetSocketAddress addr) {
     Collection<Integer> ports = addrs.get(addr.getAddress());
@@ -56,7 +56,7 @@ public class HostSet implements Iterable<InetSocketAddress> {
 
   /**
    * The function that checks whether there exists an entry foo in the set
-   * so that addr <= foo.
+   * so that addr &lt;= foo.
    */
   boolean match(InetSocketAddress addr) {
     int port = addr.getPort();

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

@@ -29,6 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports;
 import org.apache.hadoop.util.Timer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -82,7 +83,7 @@ public class SlowPeerTracker {
 
   /**
    * Information about peers that have reported a node as being slow.
-   * Each outer map entry is a map of (DatanodeId) -> (timestamp),
+   * Each outer map entry is a map of (DatanodeId) {@literal ->} (timestamp),
    * mapping reporting nodes to the timestamp of the last report from
    * that node.
    *
@@ -146,7 +147,7 @@ public class SlowPeerTracker {
   /**
    * Retrieve all reports for all nodes. Stale reports are excluded.
    *
-   * @return map from SlowNodeId -> (set of nodes reporting peers).
+   * @return map from SlowNodeId {@literal ->} (set of nodes reporting peers).
    */
   public Map<String, SortedSet<String>> getReportsForAllDataNodes() {
     if (allReports.isEmpty()) {

+ 36 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java

@@ -56,9 +56,9 @@ import com.google.common.collect.Lists;
  * block pool id, on this DataNode.
  * 
  * This class supports the following functionality:
- * <ol>
+ * <ul>
  * <li> Formatting a new block pool storage</li>
- * <li> Recovering a storage state to a consistent state (if possible></li>
+ * <li> Recovering a storage state to a consistent state (if possible)</li>
  * <li> Taking a snapshot of the block pool during upgrade</li>
  * <li> Rolling back a block pool to a previous snapshot</li>
  * <li> Finalizing block storage by deletion of a snapshot</li>
@@ -139,11 +139,12 @@ public class BlockPoolSliceStorage extends Storage {
 
   /**
    * Load one storage directory. Recover from previous transitions if required.
-   *
-   * @param nsInfo namespace information
-   * @param dataDir the root path of the storage directory
-   * @param startOpt startup option
-   * @return the StorageDirectory successfully loaded.
+   * @param nsInfo  namespace information
+   * @param location  the root path of the storage directory
+   * @param startOpt  startup option
+   * @param callables list of callable storage directory
+   * @param conf configuration
+   * @return
    * @throws IOException
    */
   private StorageDirectory loadStorageDirectory(NamespaceInfo nsInfo,
@@ -205,8 +206,10 @@ public class BlockPoolSliceStorage extends Storage {
    * data volume.
    *
    * @param nsInfo namespace information
-   * @param dataDirs storage directories of block pool
+   * @param location storage directories of block pool
    * @param startOpt startup option
+   * @param callables list of callable storage directory
+   * @param conf configuration
    * @return an array of loaded block pool directories.
    * @throws IOException on error
    */
@@ -240,8 +243,10 @@ public class BlockPoolSliceStorage extends Storage {
    * data volume.
    *
    * @param nsInfo namespace information
-   * @param dataDirs storage directories of block pool
+   * @param location storage directories of block pool
    * @param startOpt startup option
+   * @param callables list of callable storage directory
+   * @param conf configuration
    * @throws IOException on error
    */
   List<StorageDirectory> recoverTransitionRead(NamespaceInfo nsInfo,
@@ -348,13 +353,18 @@ public class BlockPoolSliceStorage extends Storage {
    * Analyze whether a transition of the BP state is required and
    * perform it if necessary.
    * <br>
-   * Rollback if previousLV >= LAYOUT_VERSION && prevCTime <= namenode.cTime.
-   * Upgrade if this.LV > LAYOUT_VERSION || this.cTime < namenode.cTime Regular
-   * startup if this.LV = LAYOUT_VERSION && this.cTime = namenode.cTime
+   * Rollback if:
+   * previousLV &gt;= LAYOUT_VERSION && prevCTime &lt;= namenode.cTime.
+   * Upgrade if:
+   * this.LV &gt; LAYOUT_VERSION || this.cTime &lt; namenode.cTime
+   * Regular startup if:
+   * this.LV = LAYOUT_VERSION && this.cTime = namenode.cTime
    * 
-   * @param sd storage directory <SD>/current/<bpid>
+   * @param sd storage directory @{literal <SD>/current/<bpid>}
    * @param nsInfo namespace info
    * @param startOpt startup option
+   * @param callables list of callable storage directory
+   * @param conf configuration
    * @return true if the new properties has been written.
    */
   private boolean doTransition(StorageDirectory sd, NamespaceInfo nsInfo,
@@ -416,20 +426,20 @@ public class BlockPoolSliceStorage extends Storage {
   }
 
   /**
-   * Upgrade to any release after 0.22 (0.22 included) release e.g. 0.22 => 0.23
+   * Upgrade to any release after 0.22 (0.22 included) release
+   * e.g. 0.22 =&gt; 0.23
    * Upgrade procedure is as follows:
    * <ol>
-   * <li>If <SD>/current/<bpid>/previous exists then delete it</li>
-   * <li>Rename <SD>/current/<bpid>/current to
-   * <SD>/current/bpid/current/previous.tmp</li>
-   * <li>Create new <SD>current/<bpid>/current directory</li>
-   * <ol>
+   * <li>If {@literal <SD>/current/<bpid>/previous} exists then delete it</li>
+   * <li>Rename {@literal <SD>/current/<bpid>/current} to
+   * {@literal <SD>/current/bpid/current/previous.tmp}</li>
+   * <li>Create new {@literal <SD>current/<bpid>/current} directory</li>
    * <li>Hard links for block files are created from previous.tmp to current</li>
    * <li>Save new version file in current directory</li>
+   * <li>Rename previous.tmp to previous</li>
    * </ol>
-   * <li>Rename previous.tmp to previous</li> </ol>
    * 
-   * @param bpSd storage directory <SD>/current/<bpid>
+   * @param bpSd storage directory {@literal <SD>/current/<bpid>}
    * @param nsInfo Namespace Info from the namenode
    * @throws IOException on error
    */
@@ -777,12 +787,12 @@ public class BlockPoolSliceStorage extends Storage {
   }
 
   /**
-   * Get a target subdirectory under current/ for a given block file that is being
-   * restored from trash.
+   * Get a target subdirectory under current/ for a given block file that is
+   * being restored from trash.
    *
    * The subdirectory structure under trash/ mirrors that under current/ to keep
    * implicit memory of where the files are to be restored.
-   *
+   * @param blockFile  block file that is being restored from trash.
    * @return the target directory to restore a previously deleted block file.
    */
   @VisibleForTesting
@@ -847,6 +857,7 @@ public class BlockPoolSliceStorage extends Storage {
   /**
    * Create a rolling upgrade marker file for each BP storage root, if it
    * does not exist already.
+   * @param dnStorageDirs
    */
   public void setRollingUpgradeMarkers(List<StorageDirectory> dnStorageDirs)
       throws IOException {
@@ -872,6 +883,7 @@ public class BlockPoolSliceStorage extends Storage {
    * Check whether the rolling upgrade marker file exists for each BP storage
    * root. If it does exist, then the marker file is cleared and more
    * importantly the layout upgrade is finalized.
+   * @param dnStorageDirs
    */
   public void clearRollingUpgradeMarkers(List<StorageDirectory> dnStorageDirs)
       throws IOException {

+ 10 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockRecoveryWorker.java

@@ -338,19 +338,24 @@ public class BlockRecoveryWorker {
 
   /**
    * blk_0  blk_1  blk_2  blk_3  blk_4  blk_5  blk_6  blk_7  blk_8
-   *  64k    64k    64k    64k    64k    64k    64k    64k    64k   <-- stripe_0
+   *  64k    64k    64k    64k    64k    64k    64k    64k    64k   &lt;--
+   *  stripe_0
    *  64k    64k    64k    64k    64k    64k    64k    64k    64k
-   *  64k    64k    64k    64k    64k    64k    64k    61k    <-- startStripeIdx
+   *  64k    64k    64k    64k    64k    64k    64k    61k    &lt;--
+   *  startStripeIdx
    *  64k    64k    64k    64k    64k    64k    64k
    *  64k    64k    64k    64k    64k    64k    59k
    *  64k    64k    64k    64k    64k    64k
-   *  64k    64k    64k    64k    64k    64k                <-- last full stripe
-   *  64k    64k    13k    64k    55k     3k              <-- target last stripe
+   *  64k    64k    64k    64k    64k    64k                &lt;--
+   *  last full stripe
+   *  64k    64k    13k    64k    55k     3k              &lt;--
+   *  target last stripe
    *  64k    64k           64k     1k
    *  64k    64k           58k
    *  64k    64k
    *  64k    19k
-   *  64k                                               <-- total visible stripe
+   *  64k                                               &lt;--
+   *  total visible stripe
    *
    *  Due to different speed of streamers, the internal blocks in a block group
    *  could have different lengths when the block group isn't ended normally.

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

@@ -189,7 +189,7 @@ public class BlockScanner {
   }
 
   /**
-   * Returns true if the block scanner is enabled.<p/>
+   * Returns true if the block scanner is enabled.
    *
    * If the block scanner is disabled, no volume scanners will be created, and
    * no threads will start.
@@ -234,7 +234,7 @@ public class BlockScanner {
   }
 
   /**
-   * Stops and removes a volume scanner.<p/>
+   * Stops and removes a volume scanner.
    *
    * This function will block until the volume scanner has stopped.
    *
@@ -260,7 +260,7 @@ public class BlockScanner {
   }
 
   /**
-   * Stops and removes all volume scanners.<p/>
+   * Stops and removes all volume scanners.
    *
    * This function will block until all the volume scanners have stopped.
    */

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -242,7 +242,7 @@ import org.slf4j.LoggerFactory;
  * DataNodes.
  *
  * The DataNode maintains just one critical table:
- *   block-> stream of bytes (of BLOCK_SIZE or less)
+ *   block{@literal ->} stream of bytes (of BLOCK_SIZE or less)
  *
  * This info is stored on a local disk.  The DataNode
  * reports the table's contents to the NameNode upon startup
@@ -527,7 +527,7 @@ public class DataNode extends ReconfigurableBase
   }
 
   /**
-   * {@inheritdoc}.
+   * {@inheritDoc }.
    */
   @Override
   public String reconfigurePropertyImpl(String property, String newVal)
@@ -2713,7 +2713,8 @@ public class DataNode extends ReconfigurableBase
     return locations;
   }
 
-  /** Instantiate & Start a single datanode daemon and wait for it to finish.
+  /** Instantiate &amp; Start a single datanode daemon and wait for it to
+   * finish.
    *  If this thread is specifically interrupted, it will stop waiting.
    */
   @VisibleForTesting
@@ -2722,7 +2723,8 @@ public class DataNode extends ReconfigurableBase
     return createDataNode(args, conf, null);
   }
   
-  /** Instantiate & Start a single datanode daemon and wait for it to finish.
+  /** Instantiate &amp; Start a single datanode daemon and wait for it to
+   * finish.
    *  If this thread is specifically interrupted, it will stop waiting.
    */
   @VisibleForTesting

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

@@ -215,7 +215,9 @@ public class DataStorage extends Storage {
 
   /**
    * VolumeBuilder holds the metadata (e.g., the storage directories) of the
-   * prepared volume returned from {@link prepareVolume()}. Calling {@link build()}
+   * prepared volume returned from
+   * {@link #prepareVolume(DataNode, StorageLocation, List)}.
+   * Calling {@link VolumeBuilder#build()}
    * to add the metadata to {@link DataStorage} so that this prepared volume can
    * be active.
    */

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

@@ -280,7 +280,6 @@ public class DirectoryScanner implements Runnable {
   /**
    * Create a new directory scanner, but don't cycle it running yet.
    *
-   * @param datanode the parent datanode
    * @param dataset the dataset to scan
    * @param conf the Configuration object
    */

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

@@ -178,7 +178,6 @@ public class FileIoProvider {
    * Call sync_file_range on the given file descriptor.
    *
    * @param  volume target volume. null if unavailable.
-   * @throws IOException
    */
   public void syncFileRange(
       @Nullable FsVolumeSpi volume, FileDescriptor outFd,
@@ -198,7 +197,6 @@ public class FileIoProvider {
    * Call posix_fadvise on the given file descriptor.
    *
    * @param  volume target volume. null if unavailable.
-   * @throws IOException
    */
   public void posixFadvise(
       @Nullable FsVolumeSpi volume, String identifier, FileDescriptor outFd,
@@ -394,7 +392,6 @@ public class FileIoProvider {
    * @param volume  target volume. null if unavailable.
    * @param fd  File descriptor object.
    * @return  FileOutputStream to the given file object.
-   * @throws  FileNotFoundException
    */
   public FileOutputStream getFileOutputStream(
       @Nullable FsVolumeSpi volume, FileDescriptor fd) {

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

@@ -45,8 +45,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * VolumeScanner scans a single volume.  Each VolumeScanner has its own thread.<p/>
- * They are all managed by the DataNode's BlockScanner.
+ * VolumeScanner scans a single volume.  Each VolumeScanner has its own thread.
+ * <p>They are all managed by the DataNode's BlockScanner.
  */
 public class VolumeScanner extends Thread {
   public static final Logger LOG =

+ 4 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java

@@ -24,7 +24,10 @@ import com.google.common.annotations.Beta;
 import com.google.common.annotations.GwtCompatible;
 import com.google.common.base.Preconditions;
 import static com.google.common.base.Preconditions.checkNotNull;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
 import com.google.common.util.concurrent.Uninterruptibles;
 import static java.util.concurrent.atomic.AtomicReferenceFieldUpdater
     .newUpdater;
@@ -52,9 +55,7 @@ import java.util.logging.Logger;
  * {@link ListeningExecutorService}, and deriving a {@code Future} from an
  * existing one, typically using methods like {@link Futures#transform
  * (ListenableFuture, com.google.common.base.Function) Futures.transform}
- * and {@link Futures#catching(ListenableFuture, Class,
- * com.google.common.base.Function, java.util.concurrent.Executor)
- * Futures.catching}.
+ * and its overloaded versions.
  * <p>
  * <p>This class implements all methods in {@code ListenableFuture}.
  * Subclasses should provide a way to set the result of the computation
@@ -1265,12 +1266,6 @@ public abstract class AbstractFuture<V> implements ListenableFuture<V> {
    *       r.run();
    *     }
    *   }}</pre>
-   * <p>
-   * <p>This should be preferred to {@link #newDirectExecutorService()}
-   * because implementing the {@link ExecutorService} subinterface
-   * necessitates significant performance overhead.
-   *
-   * @since 18.0
    */
   public static Executor directExecutor() {
     return DirectExecutor.INSTANCE;

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

@@ -192,7 +192,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   FsVolumeReferences getFsVolumeReferences();
 
   /**
-   * Add a new volume to the FsDataset.<p/>
+   * Add a new volume to the FsDataset.
    *
    * If the FSDataset supports block scanning, this function registers
    * the new volume with the block scanner.
@@ -226,7 +226,7 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /** @return the volume that contains a replica of the block. */
   V getVolume(ExtendedBlock b);
 
-  /** @return a volume information map (name => info). */
+  /** @return a volume information map (name {@literal =>} info). */
   Map<String, Object> getVolumeInfoMap();
 
   /**
@@ -273,7 +273,8 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
 
   /**
    * Get reference to the replica meta info in the replicasMap. 
-   * To be called from methods that are synchronized on {@link FSDataset}
+   * To be called from methods that are synchronized on
+   * implementations of {@link FsDatasetSpi}
    * @return replica from the replicas map
    */
   @Deprecated
@@ -394,7 +395,7 @@ 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 block Block to be finalized
+   * @param b Block to be finalized
    * @param fsyncDir whether to sync the directory changes to durable device.
    * @throws IOException
    * @throws ReplicaNotFoundException if the replica can not be found when the
@@ -488,14 +489,13 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
   /**
    * Determine if the specified block is cached.
    * @param bpid Block pool id
-   * @param blockIds - block id
+   * @param blockId - block id
    * @return true if the block is cached
    */
   boolean isCached(String bpid, long blockId);
 
     /**
      * Check if all the data directories are healthy
-     * @return A set of unhealthy data directories.
      * @param failedVolumes
      */
   void handleVolumeFailures(Set<FsVolumeSpi> failedVolumes);

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

@@ -98,17 +98,17 @@ public interface FsVolumeSpi
 
   /**
    * BlockIterator will return ExtendedBlock entries from a block pool in
-   * this volume.  The entries will be returned in sorted order.<p/>
+   * this volume.  The entries will be returned in sorted order.<p>
    *
    * BlockIterator objects themselves do not always have internal
    * synchronization, so they can only safely be used by a single thread at a
-   * time.<p/>
+   * time.<p>
    *
    * Closing the iterator does not save it.  You must call save to save it.
    */
   interface BlockIterator extends Closeable {
     /**
-     * Get the next block.<p/>
+     * Get the next block.<p>
      *
      * Note that this block may be removed in between the time we list it,
      * and the time the caller tries to use it, or it may represent a stale
@@ -146,7 +146,7 @@ public interface FsVolumeSpi
     void save() throws IOException;
 
     /**
-     * Set the maximum staleness of entries that we will return.<p/>
+     * Set the maximum staleness of entries that we will return.<p>
      *
      * A maximum staleness of 0 means we will never return stale entries; a
      * larger value will allow us to reduce resource consumption in exchange
@@ -211,12 +211,12 @@ public interface FsVolumeSpi
    * Because millions of these structures may be created, we try to save
    * memory here.  So instead of storing full paths, we store path suffixes.
    * The block file, if it exists, will have a path like this:
-   * <volume_base_path>/<block_path>
+   * {@literal <volume_base_path>/<block_path>}
    * So we don't need to store the volume path, since we already know what the
    * volume is.
    *
    * The metadata file, if it exists, will have a path like this:
-   * <volume_base_path>/<block_path>_<genstamp>.meta
+   * {@literal <volume_base_path>/<block_path>_<genstamp>.meta}
    * So if we have a block file, there isn't any need to store the block path
    * again.
    *
@@ -439,6 +439,7 @@ public interface FsVolumeSpi
    * @param bpid block pool id to scan
    * @param report the list onto which blocks reports are placed
    * @param reportCompiler
+   * @throws InterruptedException
    * @throws IOException
    */
   void compileReport(String bpid,

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

@@ -94,7 +94,8 @@ public class OutlierDetector {
 
   /**
    * Return a set of nodes/ disks whose latency is much higher than
-   * their counterparts. The input is a map of (resource -> aggregate latency)
+   * their counterparts. The input is a map of (resource {@literal ->} aggregate
+   * latency)
    * entries.
    *
    * The aggregate may be an arithmetic mean or a percentile e.g.

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/DiskBalancerException.java

@@ -60,7 +60,6 @@ public class DiskBalancerException extends IOException {
   /**
    * Constructs an {@code IOException} with the specified detail message and
    * cause.
-   * <p/>
    * <p> Note that the detail message associated with {@code cause} is
    * <i>not</i>
    * automatically incorporated into this exception's detail message.

+ 5 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerCluster.java

@@ -50,21 +50,20 @@ import java.util.concurrent.Future;
 
 /**
  * DiskBalancerCluster represents the nodes that we are working against.
- * <p/>
+ * <p>
  * Please Note :
- * <p/>
  * Semantics of inclusionList and exclusionLists.
- * <p/>
+ * <p>
  * If a non-empty inclusionList is specified then the diskBalancer assumes that
  * the user is only interested in processing that list of nodes. This node list
  * is checked against the exclusionList and only the nodes in inclusionList but
  * not in exclusionList is processed.
- * <p/>
+ * <p>
  * if inclusionList is empty, then we assume that all live nodes in the nodes is
  * to be processed by diskBalancer. In that case diskBalancer will avoid any
  * nodes specified in the exclusionList but will process all nodes in the
  * cluster.
- * <p/>
+ * <p>
  * In other words, an empty inclusionList is means all the nodes otherwise
  * only a given list is processed and ExclusionList is always honored.
  */
@@ -291,7 +290,7 @@ public class DiskBalancerCluster {
   /**
    * Compute plan takes a node and constructs a planner that creates a plan that
    * we would like to follow.
-   * <p/>
+   * <p>
    * This function creates a thread pool and executes a planner on each node
    * that we are supposed to plan for. Each of these planners return a NodePlan
    * that we can persist or schedule for execution with a diskBalancer

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/datamodel/DiskBalancerDataNode.java

@@ -186,7 +186,7 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
   }
 
   /**
-   * returns NodeDataDensity Metric.
+   * Returns NodeDataDensity Metric.
    *
    * @return float
    */
@@ -195,8 +195,8 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
   }
 
   /**
-   * computes nodes data density.
-   * <p/>
+   * Computes nodes data density.
+   *
    * This metric allows us to compare different  nodes and how well the data is
    * spread across a set of volumes inside the node.
    */
@@ -231,8 +231,8 @@ public class DiskBalancerDataNode implements Comparable<DiskBalancerDataNode> {
 
   /**
    * Adds a volume to the DataNode.
-   * <p/>
-   * it is assumed that we have one thread per node hence this call is not
+   *
+   * It is assumed that we have one thread per node hence this call is not
    * synchronised neither is the map is protected.
    *
    * @param volume - volume

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/diskbalancer/planner/GreedyPlanner.java

@@ -34,7 +34,7 @@ import java.util.TreeSet;
 /**
  * Greedy Planner is a simple planner that computes the largest possible move at
  * any point of time given a volumeSet.
- * <p/>
+ * <p>
  * This is done by choosing the disks with largest  amount of data above and
  * below the idealStorage and then a move is scheduled between them.
  */

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

@@ -156,7 +156,7 @@ public final class AclStorage {
    *
    * @param inode INode to read
    * @param snapshotId int ID of snapshot to read
-   * @return List<AclEntry> containing extended inode ACL entries
+   * @return {@literal List<AclEntry>} containing extended inode ACL entries
    */
   public static List<AclEntry> readINodeAcl(INode inode, int snapshotId) {
     AclFeature f = inode.getAclFeature(snapshotId);
@@ -167,7 +167,7 @@ public final class AclStorage {
    * Reads the existing extended ACL entries of an INodeAttribute object.
    *
    * @param inodeAttr INode to read
-   * @return List<AclEntry> containing extended inode ACL entries
+   * @return {@code List<AclEntry>} containing extended inode ACL entries
    */
   public static List<AclEntry> readINodeAcl(INodeAttributes inodeAttr) {
     AclFeature f = inodeAttr.getAclFeature();
@@ -175,7 +175,7 @@ public final class AclStorage {
   }
 
   /**
-   * Build list of AclEntries from the AclFeature
+   * Build list of AclEntries from the {@link AclFeature}
    * @param aclFeature AclFeature
    * @return List of entries
    */
@@ -204,7 +204,7 @@ public final class AclStorage {
    * ACL modification APIs, which always apply a delta on top of current state.
    *
    * @param inode INode to read
-   * @return List<AclEntry> containing all logical inode ACL entries
+   * @return {@code List<AclEntry>} containing all logical inode ACL entries
    */
   public static List<AclEntry> readINodeLogicalAcl(INode inode) {
     FsPermission perm = inode.getFsPermission();
@@ -262,7 +262,7 @@ public final class AclStorage {
    * {@link AclFeature}.
    *
    * @param inode INode to update
-   * @param newAcl List<AclEntry> containing new ACL entries
+   * @param newAcl {@code List<AclEntry>} containing new ACL entries
    * @param snapshotId int latest snapshot ID of inode
    * @throws AclException if the ACL is invalid for the given inode
    * @throws QuotaExceededException if quota limit is exceeded
@@ -312,8 +312,8 @@ public final class AclStorage {
   /**
    * Creates an AclFeature from the given ACL entries.
    *
-   * @param accessEntries List<AclEntry> access ACL entries
-   * @param defaultEntries List<AclEntry> default ACL entries
+   * @param accessEntries {@code List<AclEntry>} access ACL entries
+   * @param defaultEntries {@code List<AclEntry>} default ACL entries
    * @return AclFeature containing the required ACL entries
    */
   private static AclFeature createAclFeature(List<AclEntry> accessEntries,
@@ -347,7 +347,7 @@ public final class AclStorage {
    * POSIX ACLs model, which presents the mask as the permissions of the group
    * class.
    *
-   * @param accessEntries List<AclEntry> access ACL entries
+   * @param accessEntries {@code List<AclEntry>} access ACL entries
    * @param existingPerm FsPermission existing permissions
    * @return FsPermission new permissions
    */
@@ -365,7 +365,7 @@ public final class AclStorage {
    * group and other permissions are in order.  Also preserve sticky bit and
    * toggle ACL bit off.
    *
-   * @param accessEntries List<AclEntry> access ACL entries
+   * @param accessEntries {@code List<AclEntry>} access ACL entries
    * @param existingPerm FsPermission existing permissions
    * @return FsPermission new permissions
    */

+ 28 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -67,7 +67,7 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants
 
 /**
  * Manages the list of encryption zones in the filesystem.
- * <p/>
+ * <p>
  * The EncryptionZoneManager has its own lock, but relies on the FSDirectory
  * lock being held for many operations. The FSDirectory lock should not be
  * taken if the manager lock is already held.
@@ -294,7 +294,7 @@ public class EncryptionZoneManager {
 
   /**
    * Add a new encryption zone.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    *
    * @param inodeId of the encryption zone
@@ -308,7 +308,7 @@ public class EncryptionZoneManager {
 
   /**
    * Add a new encryption zone.
-   * <p/>
+   * <p>
    * Does not assume that the FSDirectory lock is held.
    *
    * @param inodeId of the encryption zone
@@ -326,7 +326,7 @@ public class EncryptionZoneManager {
 
   /**
    * Remove an encryption zone.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   void removeEncryptionZone(Long inodeId) {
@@ -344,7 +344,7 @@ public class EncryptionZoneManager {
 
   /**
    * Returns true if an IIP is within an encryption zone.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   boolean isInAnEZ(INodesInPath iip) throws UnresolvedLinkException,
@@ -355,7 +355,7 @@ public class EncryptionZoneManager {
 
   /**
    * Returns the full path from an INode id.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   String getFullPathName(Long nodeId) {
@@ -370,7 +370,7 @@ public class EncryptionZoneManager {
   /**
    * Get the key name for an encryption zone. Returns null if <tt>iip</tt> is
    * not within an encryption zone.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   String getKeyName(final INodesInPath iip) throws IOException {
@@ -385,7 +385,7 @@ public class EncryptionZoneManager {
   /**
    * Looks up the EncryptionZoneInt for a path within an encryption zone.
    * Returns null if path is not within an EZ.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip)
@@ -434,7 +434,7 @@ public class EncryptionZoneManager {
    * Looks up the nearest ancestor EncryptionZoneInt that contains the given
    * path (excluding itself).
    * Returns null if path is not within an EZ, or the path is the root dir '/'
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   private EncryptionZoneInt getParentEncryptionZoneForPath(INodesInPath iip)
@@ -467,7 +467,7 @@ public class EncryptionZoneManager {
   /**
    * Throws an exception if the provided path cannot be renamed into the
    * destination because of differing parent encryption zones.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    *
    * @param srcIIP source IIP
@@ -529,7 +529,7 @@ public class EncryptionZoneManager {
 
   /**
    * Create a new encryption zone.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   XAttr createEncryptionZone(INodesInPath srcIIP, CipherSuite suite,
@@ -573,7 +573,7 @@ public class EncryptionZoneManager {
 
   /**
    * Cursor-based listing of encryption zones.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
    */
   BatchedListEntries<EncryptionZone> listEncryptionZones(long prevId)
@@ -621,6 +621,8 @@ public class EncryptionZoneManager {
    * @param zoneId
    * @param zonePath
    * @return true if path resolve to the id, false if not.
+   * @throws AccessControlException
+   * @throws ParentNotDirectoryException
    * @throws UnresolvedLinkException
    */
   private boolean pathResolvesToId(final long zoneId, final String zonePath)
@@ -645,6 +647,9 @@ public class EncryptionZoneManager {
   /**
    * Re-encrypts the given encryption zone path. If the given path is not the
    * root of an encryption zone, an exception is thrown.
+   * @param zoneIIP
+   * @param keyVersionName
+   * @throws IOException
    */
   List<XAttr> reencryptEncryptionZone(final INodesInPath zoneIIP,
       final String keyVersionName) throws IOException {
@@ -673,7 +678,9 @@ public class EncryptionZoneManager {
   /**
    * Cancels the currently-running re-encryption of the given encryption zone.
    * If the given path is not the root of an encryption zone,
-   * * an exception is thrown.
+   * an exception is thrown.
+   * @param zoneIIP
+   * @throws IOException
    */
   List<XAttr> cancelReencryptEncryptionZone(final INodesInPath zoneIIP)
       throws IOException {
@@ -693,8 +700,10 @@ public class EncryptionZoneManager {
 
   /**
    * Cursor-based listing of zone re-encryption status.
-   * <p/>
+   * <p>
    * Called while holding the FSDirectory lock.
+   * @param prevId
+   * @throws IOException
    */
   BatchedListEntries<ZoneReencryptionStatus> listReencryptionStatus(
       final long prevId) throws IOException {
@@ -735,6 +744,10 @@ public class EncryptionZoneManager {
 
   /**
    * Return whether an INode is an encryption zone root.
+   * @param inode
+   * @param name
+   * @return true when INode is an encryption zone root else false
+   * @throws FileNotFoundException
    */
   boolean isEncryptionZoneRoot(final INode inode, final String name)
       throws FileNotFoundException {
@@ -756,6 +769,7 @@ public class EncryptionZoneManager {
    * Return whether an INode is an encryption zone root.
    *
    * @param inode the zone inode
+   * @param name
    * @throws IOException if the inode is not a directory,
    *                     or is a directory but not the root of an EZ.
    */

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

@@ -634,12 +634,10 @@ public class FSDirectory implements Closeable {
    *            no permission checks.
    * @param src The path to resolve.
    * @param dirOp The {@link DirOp} that controls additional checks.
-   * @param resolveLink If false, only ancestor symlinks will be checked.  If
-   *         true, the last inode will also be checked.
    * @return if the path indicates an inode, return path after replacing up to
-   *         <inodeid> with the corresponding path of the inode, else the path
-   *         in {@code src} as is. If the path refers to a path in the "raw"
-   *         directory, return the non-raw pathname.
+   *        {@code <inodeid>} with the corresponding path of the inode, else
+   *        the path in {@code src} as is. If the path refers to a path in
+   *        the "raw" directory, return the non-raw pathname.
    * @throws FileNotFoundException
    * @throws AccessControlException
    * @throws ParentNotDirectoryException

+ 19 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -341,10 +341,11 @@ import org.slf4j.LoggerFactory;
  *
  * This class and its contents keep:
  *
- * 1)  Valid fsname --> blocklist  (kept on disk, logged)
+ * 1)  Valid fsname {@literal -->} blocklist  (kept on disk, logged)
  * 2)  Set of all valid blocks (inverted #1)
- * 3)  block --> machinelist (kept in memory, rebuilt dynamically from reports)
- * 4)  machine --> blocklist (inverted #2)
+ * 3)  block {@literal -->} machinelist (kept in memory, rebuilt dynamically
+ *     from reports)
+ * 4)  machine {@literal -->} blocklist (inverted #2)
  * 5)  LRU cache of updated-heartbeat machines
  */
 @InterfaceAudience.Private
@@ -1732,11 +1733,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * return a list of blocks & their locations on <code>datanode</code> whose
-   * total size is <code>size</code>
+   * return a list of blocks &amp; their locations on {@code datanode} whose
+   * total size is {@code size}
    *
    * @param datanode on which blocks are located
    * @param size total size of blocks
+   * @param minimumBlockSize
    */
   public BlocksWithLocations getBlocks(DatanodeID datanode, long size, long
       minimumBlockSize) throws IOException {
@@ -1753,6 +1755,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Dump all metadata into specified file
+   * @param filename
    */
   void metaSave(String filename) throws IOException {
     String operationName = "metaSave";
@@ -1884,6 +1887,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /////////////////////////////////////////////////////////
   /**
    * Set permissions for an existing file.
+   * @param src
+   * @param permission
    * @throws IOException
    */
   void setPermission(String src, FsPermission permission) throws IOException {
@@ -1908,6 +1913,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Set owner for an existing file.
+   * @param src
+   * @param group
+   * @param username
    * @throws IOException
    */
   void setOwner(String src, String username, String group)
@@ -2188,6 +2196,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param replication new replication
    * @return true if successful; 
    *         false if file does not exist or is a directory
+   * @throws  IOException
    */
   boolean setReplication(final String src, final short replication)
       throws IOException {
@@ -2219,6 +2228,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *
    * @param src file/directory path
    * @param policyName storage policy name
+   * @throws  IOException
    */
   void setStoragePolicy(String src, String policyName) throws IOException {
     final String operationName = "setStoragePolicy";
@@ -2245,6 +2255,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * Satisfy the storage policy for a file or a directory.
    *
    * @param src file/directory path
+   * @throws  IOException
    */
   void satisfyStoragePolicy(String src, boolean logRetryCache)
       throws IOException {
@@ -2295,6 +2306,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * unset storage policy set for a given file or a directory.
    *
    * @param src file/directory path
+   * @throws  IOException
    */
   void unsetStoragePolicy(String src) throws IOException {
     final String operationName = "unsetStoragePolicy";
@@ -2321,6 +2333,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param src
    *          file/directory path
    * @return storage policy object
+   * @throws  IOException
    */
   BlockStoragePolicy getStoragePolicy(String src) throws IOException {
     checkOperation(OperationCategory.READ);
@@ -2336,6 +2349,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * @return All the existing block storage policies
+   * @throws  IOException
    */
   BlockStoragePolicy[] getStoragePolicies() throws IOException {
     checkOperation(OperationCategory.READ);

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

@@ -523,8 +523,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
    * 2. For a {@link WithName} node, since the node must be in a snapshot, we 
    * only count the quota usage for those nodes that still existed at the 
    * creation time of the snapshot associated with the {@link WithName} node.
-   * We do not count in the size of the diff list.  
-   * <pre>
+   * We do not count in the size of the diff list.
+   * </pre>
    *
    * @param bsps Block storage policy suite to calculate intended storage type usage
    * @param blockStoragePolicyId block storage policy id of the current INode

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

@@ -669,10 +669,10 @@ public abstract class INodeReference extends INode {
     
     /**
      * {@inheritDoc}
-     * <br/>
+     * <br>
      * To destroy a DstReference node, we first remove its link with the 
-     * referred node. If the reference number of the referred node is <= 0, we 
-     * destroy the subtree of the referred node. Otherwise, we clean the 
+     * referred node. If the reference number of the referred node is &lt;= 0,
+     * we destroy the subtree of the referred node. Otherwise, we clean the
      * referred node's subtree and delete everything created after the last 
      * rename operation, i.e., everything outside of the scope of the prior 
      * WithName nodes.

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

@@ -329,8 +329,8 @@ public class INodesInPath {
   }
 
   /**
-   * @return the i-th inode if i >= 0;
-   *         otherwise, i < 0, return the (length + i)-th inode.
+   * @return the i-th inode if i {@literal >=} 0;
+   *         otherwise, i {@literal <} 0, return the (length + i)-th inode.
    */
   public INode getINode(int i) {
     return inodes[(i < 0) ? inodes.length + i : i];

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

@@ -112,7 +112,7 @@ public interface JournalManager extends Closeable, FormatConfirmable,
   void doRollback() throws IOException;
 
   /**
-   * Discard the segments whose first txid is >= the given txid.
+   * Discard the segments whose first txid is {@literal >=} the given txid.
    * @param startTxId The given txid should be right at the segment boundary, 
    * i.e., it should be the first txid of some segment, if segment corresponding
    * to the txid exists.

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

@@ -208,7 +208,7 @@ public class LeaseManager {
    * read or write lock.
    *
    * @param ancestorDir the ancestor {@link INodeDirectory}
-   * @return Set<INodesInPath>
+   * @return {@code Set<INodesInPath>}
    */
   public Set<INodesInPath> getINodeWithLeases(final INodeDirectory
       ancestorDir) throws IOException {

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

@@ -52,7 +52,7 @@ public final class MetaRecoveryContext  {
    * Display a prompt to the user and get his or her choice.
    *  
    * @param prompt      The prompt to display
-   * @param default     First choice (will be taken if autoChooseDefault is
+   * @param firstChoice First choice (will be taken if autoChooseDefault is
    *                    true)
    * @param choices     Other choies
    *

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

@@ -176,8 +176,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_BACKOFF_ENABLE_DE
  * is a second backup/failover NameNode, or when using federated NameNodes.)
  *
  * The NameNode controls two critical tables:
- *   1)  filename->blocksequence (namespace)
- *   2)  block->machinelist ("inodes")
+ *   1)  filename{@literal ->}blocksequence (namespace)
+ *   2)  block{@literal ->}machinelist ("inodes")
  *
  * The first table is stored on disk and is very precious.
  * The second table is rebuilt every time the NameNode comes up.
@@ -1111,7 +1111,7 @@ public class NameNode extends ReconfigurableBase implements
   }
 
   /**
-   * @return NameNodeHttpServer, used by unit tests to ensure a full shutdown,
+   * NameNodeHttpServer, used by unit tests to ensure a full shutdown,
    * so that no bind exception is thrown during restart.
    */
   @VisibleForTesting

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

@@ -94,14 +94,13 @@ import com.google.common.annotations.VisibleForTesting;
  * <p>The tool scans all files and directories, starting from an indicated
  *  root path. The following abnormal conditions are detected and handled:</p>
  * <ul>
- * <li>files with blocks that are completely missing from all datanodes.<br/>
+ * <li>files with blocks that are completely missing from all datanodes.<br>
  * In this case the tool can perform one of the following actions:
  *  <ul>
- *      <li>none ({@link #FIXING_NONE})</li>
  *      <li>move corrupted files to /lost+found directory on DFS
- *      ({@link #FIXING_MOVE}). Remaining data blocks are saved as a
+ *      ({@link #doMove}). Remaining data blocks are saved as a
  *      block chains, representing longest consecutive series of valid blocks.</li>
- *      <li>delete corrupted files ({@link #FIXING_DELETE})</li>
+ *      <li>delete corrupted files ({@link #doDelete})</li>
  *  </ul>
  *  </li>
  *  <li>detect files with under-replicated or over-replicated blocks</li>
@@ -201,7 +200,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    */
   NamenodeFsck(Configuration conf, NameNode namenode,
       NetworkTopology networktopology,
-      Map<String,String[]> pmap, PrintWriter out,
+      Map<String, String[]> pmap, PrintWriter out,
       int totalDatanodes, InetAddress remoteAddress) {
     this.conf = conf;
     this.namenode = namenode;

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

@@ -47,7 +47,7 @@ public enum Quota {
 
   /**
    * Is quota violated?
-   * The quota is violated if quota is set and usage > quota. 
+   * The quota is violated if quota is set and usage &gt; quota.
    */
   public static boolean isViolated(final long quota, final long usage) {
     return quota >= 0 && usage > quota;
@@ -55,7 +55,8 @@ public enum Quota {
 
   /**
    * Is quota violated?
-   * The quota is violated if quota is set, delta > 0 and usage + delta > quota.
+   * The quota is violated if quota is set, delta &gt; 0 and
+   * usage + delta &gt; quota.
    */
   static boolean isViolated(final long quota, final long usage,
       final long delta) {

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

@@ -319,7 +319,7 @@ public class ReencryptionHandler implements Runnable {
   /**
    * Main loop. It takes at most 1 zone per scan, and executes until the zone
    * is completed.
-   * {@see #reencryptEncryptionZoneInt(Long)}.
+   * {@link #reencryptEncryptionZone(long)}.
    */
   @Override
   public void run() {

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

@@ -31,7 +31,7 @@ import com.google.common.base.Preconditions;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
 
 /**
- * There are four types of extended attributes <XAttr> defined by the
+ * There are four types of extended attributes &lt;XAttr&gt; defined by the
  * following namespaces:
  * <br>
  * USER - extended user attributes: these can be assigned to files and
@@ -56,7 +56,7 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_
  *   is called on a file or directory in the /.reserved/raw HDFS directory
  *   hierarchy. These attributes can only be accessed by the user who have
  *   read access.
- * </br>
+ * <br>
  */
 @InterfaceAudience.Private
 public class XAttrPermissionFilter {

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

@@ -33,7 +33,7 @@ public class XAttrStorage {
 
   /**
    * Reads the extended attribute of an inode by name with prefix.
-   * <p/>
+   * <p>
    *
    * @param inode INode to read
    * @param snapshotId the snapshotId of the requested path
@@ -48,11 +48,11 @@ public class XAttrStorage {
 
   /**
    * Reads the existing extended attributes of an inode.
-   * <p/>
+   * <p>
    * Must be called while holding the FSDirectory read lock.
    *
    * @param inodeAttr INodeAttributes to read.
-   * @return List<XAttr> <code>XAttr</code> list.
+   * @return {@code XAttr} list.
    */
   public static List<XAttr> readINodeXAttrs(INodeAttributes inodeAttr) {
     XAttrFeature f = inodeAttr.getXAttrFeature();
@@ -61,7 +61,7 @@ public class XAttrStorage {
   
   /**
    * Update xattrs of inode.
-   * <p/>
+   * <p>
    * Must be called while holding the FSDirectory write lock.
    * 
    * @param inode INode to update

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/AbstractINodeDiffList.java

@@ -157,10 +157,10 @@ abstract class AbstractINodeDiffList<N extends INode,
   
   /**
    * Find the latest snapshot before a given snapshot.
-   * @param anchorId The returned snapshot's id must be <= or < this given 
-   *                 snapshot id.
-   * @param exclusive True means the returned snapshot's id must be < the given
-   *                  id, otherwise <=.
+   * @param anchorId The returned snapshot's id must be &lt;= or &lt; this
+   *                 given snapshot id.
+   * @param exclusive True means the returned snapshot's id must be &lt; the
+   *                  given id, otherwise &lt;=.
    * @return The id of the latest snapshot before the given snapshot.
    */
   public final int getPrior(int anchorId, boolean exclusive) {

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

@@ -44,15 +44,15 @@ import java.util.Objects;
  * and created after a particular snapshot and before the next snapshot. The
  * sequence will look like this:
  * <p>
- * s0->s1->s2->s3->s4->s5->s6->s7->s8->s9.
+ * {@literal s0->s1->s2->s3->s4->s5->s6->s7->s8->s9}.
  * <p>
  * Assuming a skip interval of 3, which means a new diff will be added at a
  * level higher than the current level after we have  ore than 3 snapshots.
  * Next level promotion happens after 9 snapshots and so on.
  * <p>
- * level 2:   s08------------------------------->s9
- * level 1:   S02------->s35-------->s68-------->s9
- * level 0:  s0->s1->s2->s3->s4->s5->s6->s7->s8->s9
+ * level 2:   {@literal s08------------------------------->s9}
+ * level 1:   {@literal S02------->s35-------->s68-------->s9}
+ * level 0:  {@literal s0->s1->s2->s3->s4->s5->s6->s7->s8->s9}
  * <p>
  * s02 will be created by combining diffs for s0, s1, s2 once s3 gets created.
  * Similarly, s08 will be created by combining s02, s35 and s68 once s9 gets
@@ -143,6 +143,7 @@ public class DiffListBySkipList implements DiffList<DirectoryDiff> {
      * and level.
      *
      * @param diff The element to be stored in the node.
+     * @param level
      */
     SkipListNode(DirectoryDiff diff, int level) {
       this.diff = diff;

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

@@ -106,8 +106,8 @@ public class BlockStorageMovementNeeded {
    * Add the itemInfo to tracking list for which storage movement expected if
    * necessary.
    *
-   * @param itemInfoList
-   *          - List of child in the directory
+   * @param itemInfo
+   *          - child in the directory
    * @param scanCompleted
    *          -Indicates whether the ItemInfo start id directory has no more
    *          elements to scan.
@@ -191,7 +191,6 @@ public class BlockStorageMovementNeeded {
   /**
    * Clean all the movements in spsDirsToBeTraveresed/storageMovementNeeded
    * and notify to clean up required resources.
-   * @throws IOException
    */
   public synchronized void clearQueuesWithNotification() {
     // Remove xAttr from directories

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

@@ -69,7 +69,7 @@ public class DatanodeCacheManager {
 
   /**
    * Returns the live datanodes and its storage details, which has available
-   * space (> 0) to schedule block moves. This will return array of datanodes
+   * space (&gt; 0) to schedule block moves. This will return array of datanodes
    * from its local cache. It has a configurable refresh interval in millis and
    * periodically refresh the datanode cache by fetching latest
    * {@link Context#getLiveDatanodeStorageReport()} once it elapsed refresh

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/sps/StoragePolicySatisfyManager.java

@@ -39,7 +39,7 @@ import com.google.common.annotations.VisibleForTesting;
  * configured by the administrator.
  *
  * <p>
- * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
+ * If the configured mode is {@link StoragePolicySatisfierMode#EXTERNAL}, then
  * it won't do anything, just maintains the sps invoked path ids. Administrator
  * requires to start external sps service explicitly, to fetch the sps invoked
  * path ids from namenode, then do necessary computations and block movement in
@@ -48,7 +48,7 @@ import com.google.common.annotations.VisibleForTesting;
  * external sps service functionality.
  *
  * <p>
- * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then it
+ * If the configured mode is {@link StoragePolicySatisfierMode#NONE}, then it
  * will disable the sps feature completely by clearing all queued up sps path's
  * hint.
  *
@@ -88,12 +88,12 @@ public class StoragePolicySatisfyManager {
    * This function will do following logic based on the configured sps mode:
    *
    * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
+   * If the configured mode is {@link StoragePolicySatisfierMode#EXTERNAL}, then
    * it won't do anything. Administrator requires to start external sps service
    * explicitly.
    *
    * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then the
+   * If the configured mode is {@link StoragePolicySatisfierMode#NONE}, then the
    * service is disabled and won't do any action.
    */
   public void start() {
@@ -121,12 +121,12 @@ public class StoragePolicySatisfyManager {
    * This function will do following logic based on the configured sps mode:
    *
    * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.EXTERNAL}, then
+   * If the configured mode is {@link StoragePolicySatisfierMode#EXTERNAL}, then
    * it won't do anything. Administrator requires to stop external sps service
    * explicitly, if needed.
    *
    * <p>
-   * If the configured mode is {@link StoragePolicySatisfierMode.NONE}, then the
+   * If the configured mode is {@link StoragePolicySatisfierMode#NONE}, then the
    * service is disabled and won't do any action.
    */
   public void stop() {
@@ -225,6 +225,7 @@ public class StoragePolicySatisfyManager {
 
   /**
    * Verify that satisfier queue limit exceeds allowed outstanding limit.
+   * @throws IOException
    */
   public void verifyOutstandingPathQLimit() throws IOException {
     long size = pathsToBeTraveresed.size();
@@ -269,6 +270,7 @@ public class StoragePolicySatisfyManager {
 
   /**
    * Adds the sps path to SPSPathIds list.
+   * @param id
    */
   public void addPathId(long id) {
     synchronized (pathsToBeTraveresed) {

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

@@ -179,7 +179,7 @@ public class StartupProgressView {
   /**
    * Returns all phases.
    * 
-   * @return Iterable<Phase> containing all phases
+   * @return {@code Iterable<Phase>} containing all phases
    */
   public Iterable<Phase> getPhases() {
     return EnumSet.allOf(Phase.class);
@@ -189,7 +189,7 @@ public class StartupProgressView {
    * Returns all steps within a phase.
    * 
    * @param phase Phase to get
-   * @return Iterable<Step> all steps
+   * @return {@code Iterable<Step>} all steps
    */
   public Iterable<Step> getSteps(Phase phase) {
     return new TreeSet<Step>(phases.get(phase).steps.keySet());

+ 13 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/metrics/TopMetrics.java

@@ -47,22 +47,22 @@ import static org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowMan
 
 /**
  * The interface to the top metrics.
- * <p/>
+ * <p>
  * Metrics are collected by a custom audit logger, {@link org.apache.hadoop
  * .hdfs.server.namenode.top.TopAuditLogger}, which calls TopMetrics to
  * increment per-operation, per-user counts on every audit log call. These
  * counts are used to show the top users by NameNode operation as well as
  * across all operations.
- * <p/>
+ * <p>
  * TopMetrics maintains these counts for a configurable number of time
  * intervals, e.g. 1min, 5min, 25min. Each interval is tracked by a
  * RollingWindowManager.
- * <p/>
+ * <p>
  * These metrics are published as a JSON string via {@link org.apache.hadoop
  * .hdfs.server .namenode.metrics.FSNamesystemMBean#getTopWindows}. This is
  * done by calling {@link org.apache.hadoop.hdfs.server.namenode.top.window
  * .RollingWindowManager#snapshot} on each RollingWindowManager.
- * <p/>
+ * <p>
  * Thread-safe: relies on thread-safety of RollingWindowManager
  */
 @InterfaceAudience.Private
@@ -119,6 +119,13 @@ public class TopMetrics implements MetricsSource {
    * log file. This is to be consistent when {@link TopMetrics} is charged with
    * data read back from log files instead of being invoked directly by the
    * FsNamesystem
+   * @param succeeded
+   * @param userName
+   * @param addr
+   * @param cmd
+   * @param src
+   * @param dst
+   * @param status
    */
   public void report(boolean succeeded, String userName, InetAddress addr,
       String cmd, String src, String dst, FileStatus status) {
@@ -147,6 +154,8 @@ public class TopMetrics implements MetricsSource {
    * {@link org.apache.hadoop.metrics2.MetricsRecord}s for consumption by
    * external metrics systems. Each metrics record added corresponds to the
    * reporting period a.k.a window length of the configured rolling windows.
+   * @param collector
+   * @param all
    */
   @Override
   public void getMetrics(MetricsCollector collector, boolean all) {

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindow.java

@@ -29,23 +29,24 @@ import org.slf4j.LoggerFactory;
  * Events are reported based on occurrence time. The total number of events in
  * the last period covered by the rolling window can be retrieved by the
  * {@link #getSum(long)} method.
- * <p/>
+ * <p>
  *
  * Assumptions:
- * <p/>
+ * <p>
  *
  * (1) Concurrent invocation of {@link #incAt} method are possible
- * <p/>
+ * <p>
  *
  * (2) The time parameter of two consecutive invocation of {@link #incAt} could
  * be in any given order
- * <p/>
+ * <p>
  *
  * (3) The buffering delays are not more than the window length, i.e., after two
  * consecutive invocation {@link #incAt(long time1, long)} and
- * {@link #incAt(long time2, long)}, time1 < time2 || time1 - time2 < windowLenMs.
+ * {@link #incAt(long time2, long)}, time1 &lt; time2 || time1 - time2 &lt;
+ * windowLenMs.
  * This assumption helps avoiding unnecessary synchronizations.
- * <p/>
+ * <p>
  *
  * Thread-safety is built in the {@link RollingWindow.Bucket}
  */
@@ -85,7 +86,7 @@ public class RollingWindow {
   /**
    * When an event occurs at the specified time, this method reflects that in
    * the rolling window.
-   * <p/>
+   * <p>
    *
    * @param time the time at which the event occurred
    * @param delta the delta that will be added to the window
@@ -153,6 +154,7 @@ public class RollingWindow {
      * performed. We do not need to update the {@link #updateTime} because as
      * long as the {@link #updateTime} belongs to the current view of the
      * rolling window, the algorithm works fine.
+     * @param delta
      */
     void inc(long delta) {
       value.addAndGet(delta);
@@ -161,7 +163,7 @@ public class RollingWindow {
 
   /**
    * Get value represented by this window at the specified time
-   * <p/>
+   * <p>
    *
    * If time lags behind the latest update time, the new updates are still
    * included in the sum

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/top/window/RollingWindowManager.java

@@ -38,7 +38,7 @@ import org.slf4j.LoggerFactory;
  * A class to manage the set of {@link RollingWindow}s. This class is the
  * interface of metrics system to the {@link RollingWindow}s to retrieve the
  * current top metrics.
- * <p/>
+ * <p>
  * Thread-safety is provided by each {@link RollingWindow} being thread-safe as
  * well as {@link ConcurrentHashMap} for the collection of them.
  */

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockStorageMovementCommand.java

@@ -30,11 +30,12 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
  *
  * Upon receiving this command, this DataNode pass the array of block movement
  * details to
- * {@link org.apache.hadoop.hdfs.server.datanode.StoragePolicySatisfyWorker}
- * service. Later, StoragePolicySatisfyWorker will schedule block movement tasks
- * for these blocks and monitors the completion of each task. After the block
- * movement attempt is finished(with success or failure) this DataNode will send
- * response back to NameNode about the block movement attempt finished details.
+ * {@link org.apache.hadoop.hdfs.server.sps.ExternalSPSBlockMoveTaskHandler}
+ * service. Later, ExternalSPSBlockMoveTaskHandler will schedule block movement
+ * tasks for these blocks and monitors the completion of each task. After the
+ * block movement attempt is finished(with success or failure) this DataNode
+ * will send response back to NameNode about the block movement attempt
+ * finished details.
  */
 public class BlockStorageMovementCommand extends DatanodeCommand {
   private final String blockPoolId;

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

@@ -158,7 +158,7 @@ public interface DatanodeProtocol {
    * {@link #blockReport(DatanodeRegistration, String, StorageBlockReport[], BlockReportContext)},
    * which is used to communicated blocks stored on disk.
    *
-   * @param            The datanode registration.
+   * @param registration The datanode registration.
    * @param poolId     The block pool ID for the blocks.
    * @param blockIds   A list of block IDs.
    * @return           The DatanodeCommand.

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

@@ -73,7 +73,7 @@ public interface NamenodeProtocol {
    * @param datanode  a data node
    * @param size      requested size
    * @param minBlockSize each block should be of this minimum Block Size
-   * @return          a list of blocks & their locations
+   * @return BlocksWithLocations a list of blocks &amp; their locations
    * @throws IOException if size is less than or equal to 0 or
   datanode does not exist
    */
@@ -183,7 +183,8 @@ public interface NamenodeProtocol {
   /**
    * Return a structure containing details about all edit logs
    * available to be fetched from the NameNode.
-   * @param sinceTxId return only logs that contain transactions >= sinceTxId
+   * @param sinceTxId return only logs that contain transactions {@literal >=}
+   * sinceTxId
    */
   @Idempotent
   public RemoteEditLogManifest getEditLogManifest(long sinceTxId)

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/ExternalSPSBlockMoveTaskHandler.java

@@ -36,8 +36,10 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;

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

@@ -50,14 +50,17 @@ import org.apache.hadoop.util.ToolRunner;
  * <p>The tool scans all files and directories, starting from an indicated
  *  root path. The following abnormal conditions are detected and handled:</p>
  * <ul>
- * <li>files with blocks that are completely missing from all datanodes.<br/>
+ * <li>files with blocks that are completely missing from all datanodes.<br>
  * In this case the tool can perform one of the following actions:
  *  <ul>
- *      <li>none ({@link org.apache.hadoop.hdfs.server.namenode.NamenodeFsck#FIXING_NONE})</li>
  *      <li>move corrupted files to /lost+found directory on DFS
- *      ({@link org.apache.hadoop.hdfs.server.namenode.NamenodeFsck#FIXING_MOVE}). Remaining data blocks are saved as a
- *      block chains, representing longest consecutive series of valid blocks.</li>
- *      <li>delete corrupted files ({@link org.apache.hadoop.hdfs.server.namenode.NamenodeFsck#FIXING_DELETE})</li>
+ *      ({@link org.apache.hadoop.hdfs.server.namenode.NamenodeFsck#doMove}).
+ *      Remaining data blocks are saved as a
+ *      block chains, representing longest consecutive series of valid blocks.
+ *      </li>
+ *      <li>delete corrupted files
+ *      ({@link org.apache.hadoop.hdfs.server.namenode.NamenodeFsck#doDelete})
+ *      </li>
  *  </ul>
  *  </li>
  *  <li>detect files with under-replicated or over-replicated blocks</li>

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

@@ -121,8 +121,8 @@ public class OfflineEditsViewer extends Configured implements Tool {
 
   /** Process an edit log using the chosen processor or visitor.
    * 
-   * @param inputFilename   The file to process
-   * @param outputFilename  The output file name
+   * @param inputFileName   The file to process
+   * @param outputFileName  The output file name
    * @param processor       If visitor is null, the processor to use
    * @param visitor         If non-null, the visitor to use.
    * 

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

@@ -52,7 +52,7 @@ abstract public interface OfflineEditsVisitor {
    * Begin visiting an element that encloses another element, such as
    * the beginning of the list of blocks that comprise a file.
    *
-   * @param value Token being visited
+   * @param op Token being visited
    */
   abstract void visitOp(FSEditLogOp op)
      throws IOException;

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

@@ -50,9 +50,7 @@ public class StatisticsEditsVisitor implements OfflineEditsVisitor {
    * Create a processor that writes to the file named and may or may not
    * also output to the screen, as specified.
    *
-   * @param filename Name of file to write output to
-   * @param tokenizer Input tokenizer
-   * @param printToScreen Mirror output to screen?
+   * @param out Name of file to write output to
    */
   public StatisticsEditsVisitor(OutputStream out) throws IOException {
     this.out = new PrintWriter(new OutputStreamWriter(out, Charsets.UTF_8));

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

@@ -26,11 +26,13 @@ import org.apache.hadoop.classification.InterfaceAudience;
 /**
  * File name distribution visitor. 
  * <p>
- * It analyzes file names in fsimage and prints the following information: 
+ * It analyzes file names in fsimage and prints the following information:
+ * <ul>
  * <li>Number of unique file names</li> 
  * <li>Number file names and the corresponding number range of files that use 
  * these same names</li>
  * <li>Heap saved if the file name objects are reused</li>
+ * </ul>
  */
 @InterfaceAudience.Private
 public class NameDistributionVisitor extends TextWriterImageVisitor {

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/Diff.java

@@ -470,23 +470,23 @@ public class Diff<K, E extends Diff.Element<K>> {
    * <pre>
    * 1. For (c, 0) in the posterior diff, check the element in this diff:
    * 1.1 (c', 0)  in this diff: impossible
-   * 1.2 (0, d')  in this diff: put in c-list --> (c, d')
+   * 1.2 (0, d')  in this diff: put in c-list --&gt; (c, d')
    * 1.3 (c', d') in this diff: impossible
-   * 1.4 (0, 0)   in this diff: put in c-list --> (c, 0)
+   * 1.4 (0, 0)   in this diff: put in c-list --&gt; (c, 0)
    * This is the same logic as create(E).
    * 
    * 2. For (0, d) in the posterior diff,
-   * 2.1 (c', 0)  in this diff: remove from c-list --> (0, 0)
+   * 2.1 (c', 0)  in this diff: remove from c-list --&gt; (0, 0)
    * 2.2 (0, d')  in this diff: impossible
-   * 2.3 (c', d') in this diff: remove from c-list --> (0, d')
-   * 2.4 (0, 0)   in this diff: put in d-list --> (0, d)
+   * 2.3 (c', d') in this diff: remove from c-list --&gt; (0, d')
+   * 2.4 (0, 0)   in this diff: put in d-list --&gt; (0, d)
    * This is the same logic as delete(E).
    * 
    * 3. For (c, d) in the posterior diff,
-   * 3.1 (c', 0)  in this diff: replace the element in c-list --> (c, 0)
+   * 3.1 (c', 0)  in this diff: replace the element in c-list --&gt; (c, 0)
    * 3.2 (0, d')  in this diff: impossible
-   * 3.3 (c', d') in this diff: replace the element in c-list --> (c, d')
-   * 3.4 (0, 0)   in this diff: put in c-list and d-list --> (c, d)
+   * 3.3 (c', d') in this diff: replace the element in c-list --&gt; (c, d')
+   * 3.4 (0, 0)   in this diff: put in c-list and d-list --&gt; (c, d)
    * This is the same logic as modify(E, E).
    * </pre>
    * 

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

@@ -116,8 +116,8 @@ public class XMLUtils {
    * 
    * There are three kinds of code points in XML:
    * - Those that can be represented normally,
-   * - Those that have to be escaped (for example, & must be represented 
-   *     as &amp;)
+   * - Those that have to be escaped (for example, &amp; must be represented
+   *     as {@literal &amp;})
    * - Those that cannot be represented at all in XML.
    *
    * The built-in SAX functions will handle the first two types for us just