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

This commit is contained in:
Akira Ajisaka 2018-10-31 14:43:58 +09:00
parent e4f22b08e0
commit fac9f91b29
No known key found for this signature in database
GPG Key ID: C1EDBB9CA400FD50
65 changed files with 310 additions and 246 deletions

View File

@ -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) {

View File

@ -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:

View File

@ -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 {

View File

@ -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 {

View File

@ -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

View File

@ -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.

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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();

View File

@ -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()) {

View File

@ -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 {

View File

@ -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.

View File

@ -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.
*/

View File

@ -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

View File

@ -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.
*/

View File

@ -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
*/

View File

@ -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) {

View File

@ -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 =

View File

@ -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;

View File

@ -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);

View File

@ -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,

View File

@ -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.

View File

@ -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.

View File

@ -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

View File

@ -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

View File

@ -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.
*/

View File

@ -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
*/

View File

@ -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.
*/

View File

@ -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

View File

@ -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);

View File

@ -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

View File

@ -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.

View File

@ -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];

View File

@ -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.

View File

@ -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 {

View File

@ -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
*

View File

@ -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

View File

@ -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;

View File

@ -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) {

View File

@ -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() {

View File

@ -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 {

View File

@ -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

View File

@ -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) {

View File

@ -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;

View File

@ -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

View File

@ -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

View File

@ -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) {

View File

@ -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());

View File

@ -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) {

View File

@ -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

View File

@ -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.
*/

View File

@ -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;

View File

@ -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.

View File

@ -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)

View File

@ -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;

View File

@ -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>

View File

@ -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.
*

View File

@ -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;

View File

@ -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));

View File

@ -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 {

View File

@ -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>
*

View File

@ -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