HBASE-14890 Fix broken checkstyle tool. Some javadocs had parsing errors because of wrongly formatted HTML tags. (Apekshit)

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Apekshit 2015-11-27 13:41:46 -08:00 committed by stack
parent 2317ec583d
commit 3bac31b2a4
26 changed files with 82 additions and 119 deletions

View File

@ -198,7 +198,7 @@ public abstract class Query extends OperationWithAttributes {
}
/**
* @return Map<byte[], TimeRange> a map of column families to time ranges
* @return A map of column families to time ranges
*/
public Map<byte[], TimeRange> getColumnFamilyTimeRange() {
return this.colFamTimeRangeMap;

View File

@ -792,7 +792,7 @@ public class RpcClientImpl extends AbstractRpcClient {
}
/**
* Write the RPC header: <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>
* Write the RPC header: {@code <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>}
*/
private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
// Assemble the preamble up in a buffer first and then send it. Writing individual elements,

View File

@ -924,7 +924,7 @@ public final class RequestConverter {
}
/**
* @see {@link #buildRollWALWriterRequest()
* @see {@link #buildRollWALWriterRequest()}
*/
private static RollWALWriterRequest ROLL_WAL_WRITER_REQUEST =
RollWALWriterRequest.newBuilder().build();

View File

@ -63,7 +63,7 @@ public class ReplicationQueueInfo {
/**
* Parse dead server names from znode string servername can contain "-" such as
* "ip-10-46-221-101.ec2.internal", so we need skip some "-" during parsing for the following
* cases: 2-ip-10-46-221-101.ec2.internal,52170,1364333181125-<server name>-...
* cases: 2-ip-10-46-221-101.ec2.internal,52170,1364333181125-&lt;server name>-...
*/
private static void
extractDeadServersFromZNodeString(String deadServerListStr, List<String> result) {

View File

@ -883,12 +883,12 @@ public final class HConstants {
Pattern.compile("^coprocessor\\$([0-9]+)$", Pattern.CASE_INSENSITIVE);
/**
* <pre>
* Pattern that matches a coprocessor specification. Form is:
* <code>
*&lt;coprocessor jar file location> '|' &lt<class name> ['|' &lt;priority> ['|' &lt;arguments>]]
* </code>
* ...where arguments are <code>&lt;KEY> '=' &lt;VALUE> [,...]</code>
* <p>For example: <code>hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2</code>
* {@code <coprocessor jar file location> '|' <class name> ['|' <priority> ['|' <arguments>]]}
* where arguments are {@code <KEY> '=' <VALUE> [,...]}
* For example: {@code hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2}
* </pre>
*/
public static final Pattern CP_HTD_ATTR_VALUE_PATTERN =
Pattern.compile("(^[^\\|]*)\\|([^\\|]+)\\|[\\s]*([\\d]*)[\\s]*(\\|.*)?$");

View File

@ -42,24 +42,23 @@ public class Tag {
private int offset = 0;
private int length = 0;
// The special tag will write the length of each tag and that will be
// followed by the type and then the actual tag.
// So every time the length part is parsed we need to add + 1 byte to it to
// get the type and then get the actual tag.
/**
* The special tag will write the length of each tag and that will be
* followed by the type and then the actual tag.
* So every time the length part is parsed we need to add + 1 byte to it to
* get the type and then get the actual tag.
*/
public Tag(byte tagType, String tag) {
this(tagType, Bytes.toBytes(tag));
}
/**
* @param tagType
* @param tag
* Format for a tag :
* {@code <length of tag - 2 bytes><type code - 1 byte><tag>} tag length is serialized
* using 2 bytes only but as this will be unsigned, we can have max tag length of
* (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length.
*/
public Tag(byte tagType, byte[] tag) {
/**
* Format for a tag : <length of tag - 2 bytes><type code - 1 byte><tag> taglength is serialized
* using 2 bytes only but as this will be unsigned, we can have max taglength of
* (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length.
*/
int tagLength = tag.length + TYPE_LENGTH_SIZE;
if (tagLength > MAX_TAG_LENGTH) {
throw new IllegalArgumentException(
@ -78,10 +77,7 @@ public class Tag {
* <code>bytes</code> content starting at <code>offset</code> is formatted as
* a Tag blob.
* The bytes to include the tag type, tag length and actual tag bytes.
* @param bytes
* byte array
* @param offset
* offset to start of Tag
* @param offset offset to start of Tag
*/
public Tag(byte[] bytes, int offset) {
this(bytes, offset, getLength(bytes, offset));
@ -95,12 +91,6 @@ public class Tag {
* Creates a Tag from the specified byte array, starting at offset, and for length
* <code>length</code>. Presumes <code>bytes</code> content starting at <code>offset</code> is
* formatted as a Tag blob.
* @param bytes
* byte array
* @param offset
* offset to start of the Tag
* @param length
* length of the Tag
*/
public Tag(byte[] bytes, int offset, int length) {
if (length > MAX_TAG_LENGTH) {

View File

@ -154,12 +154,10 @@ public final class Constraints {
/**
* Get the kv {@link Entry} in the descriptor for the specified class
*
* @param desc
* {@link HTableDescriptor} to read
* @param clazz
* to search for
* @return the {@link Pair} of <key, value> in the table, if that class is
* present. <tt>null</tt> otherwise.
* @param desc {@link HTableDescriptor} to read
* @param clazz To search for
* @return The {@link Pair} of {@literal <key, value>} in the table, if that class is
* present. {@code NULL} otherwise.
*/
private static Pair<String, String> getKeyValueForClass(
HTableDescriptor desc, Class<? extends Constraint> clazz) {

View File

@ -806,7 +806,7 @@ public class BucketCache implements BlockCache, HeapSize {
* Process all that are passed in even if failure being sure to remove from ramCache else we'll
* never undo the references and we'll OOME.
* @param entries Presumes list passed in here will be processed by this invocation only. No
* interference expected.
* interference expected.
* @throws InterruptedException
*/
@VisibleForTesting
@ -911,23 +911,23 @@ public class BucketCache implements BlockCache, HeapSize {
}
/**
* Blocks until elements available in <code>q</code> then tries to grab as many as possible
* Blocks until elements available in {@code q} then tries to grab as many as possible
* before returning.
* @param recepticle Where to stash the elements taken from queue. We clear before we use it
* just in case.
* @param receptacle Where to stash the elements taken from queue. We clear before we use it
* just in case.
* @param q The queue to take from.
* @return <code>receptical laden with elements taken from the queue or empty if none found.
* @return {@code receptacle} laden with elements taken from the queue or empty if none found.
*/
@VisibleForTesting
static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
final List<RAMQueueEntry> receptical)
final List<RAMQueueEntry> receptacle)
throws InterruptedException {
// Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
// ok even if list grew to accommodate thousands.
receptical.clear();
receptical.add(q.take());
q.drainTo(receptical);
return receptical;
receptacle.clear();
receptacle.add(q.take());
q.drainTo(receptacle);
return receptacle;
}
private void persistToFile() throws IOException {

View File

@ -521,8 +521,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
}
/**
* @return A Multimap<startkey, LoadQueueItem> that groups LQI by likely
* bulk load region targets.
* @return A map that groups LQI by likely bulk load region targets.
*/
private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final Table table,
ExecutorService pool, Deque<LoadQueueItem> queue,

View File

@ -143,10 +143,7 @@ public class ServerManager {
private final Map<ServerName, AdminService.BlockingInterface> rsAdmins =
new HashMap<ServerName, AdminService.BlockingInterface>();
/**
* List of region servers <ServerName> that should not get any more new
* regions.
*/
/** List of region servers that should not get any more new regions. */
private final ArrayList<ServerName> drainingServers =
new ArrayList<ServerName>();

View File

@ -399,8 +399,7 @@ public class SplitLogManager {
* It removes recovering regions under /hbase/recovering-regions/[encoded region name] so that the
* region server hosting the region can allow reads to the recovered region
* @param serverNames servers which are just recovered
* @param isMetaRecovery whether current recovery is for the meta region on
* <code>serverNames<code>
* @param isMetaRecovery whether current recovery is for the meta region on {@code serverNames}
*/
private void removeRecoveringRegions(final Set<ServerName> serverNames, Boolean isMetaRecovery) {
if (!isLogReplaying()) {

View File

@ -35,8 +35,8 @@ import org.apache.zookeeper.KeeperException;
/**
* The mapper of a sweep job.
* Takes the rows from the table and their results and map to <filename:Text, mobValue:KeyValue>
* where mobValue is the actual cell in HBase.
* Takes the rows from the table and their results and map to {@literal <filename:Text,
* mobValue:KeyValue>} where mobValue is the actual cell in HBase.
*/
@InterfaceAudience.Private
public class SweepMapper extends TableMapper<Text, KeyValue> {

View File

@ -157,7 +157,7 @@ public class TaskMonitor {
/**
* This class encapsulates an object as well as a weak reference to a proxy
* that passes through calls to that object. In art form:
* <code>
* <pre>
* Proxy <------------------
* | \
* v \
@ -166,7 +166,7 @@ public class TaskMonitor {
* MonitoredTaskImpl /
* | /
* StatAndWeakRefProxy ------/
*
* </pre>
* Since we only return the Proxy to the creator of the MonitorableStatus,
* this means that they can leak that object, and we'll detect it
* since our weak reference will go null. But, we still have the actual

View File

@ -5234,7 +5234,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* Determines whether multiple column families are present
* Precondition: familyPaths is not null
*
* @param familyPaths List of Pair<byte[] column family, String hfilePath>
* @param familyPaths List of (column family, hfilePath)
*/
private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
boolean multipleFamilies = false;
@ -5947,10 +5947,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/**
* This function is to maintain backward compatibility for 0.94 filters. HBASE-6429 combines
* both filterRow & filterRow(List<KeyValue> kvs) functions. While 0.94 code or older, it may
* not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only returns
* true when filterRow(List<KeyValue> kvs) is overridden not the filterRow(). Therefore, the
* filterRow() will be skipped.
* both filterRow & filterRow({@code List<KeyValue> kvs}) functions. While 0.94 code or older,
* it may not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only
* returns true when filterRow({@code List<KeyValue> kvs}) is overridden not the filterRow().
* Therefore, the filterRow() will be skipped.
*/
private boolean filterRow() throws IOException {
// when hasFilterRow returns true, filter.filterRow() will be called automatically inside
@ -6950,9 +6950,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
/**
* @param cell
* @param tags
* @return The passed-in List<Tag> but with the tags from <code>cell</code> added.
* @return The passed-in {@code tags} but with the tags from {@code cell} added.
*/
private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
if (cell.getTagsLength() <= 0) return tags;
@ -6965,12 +6963,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
/**
* Run a Get against passed in <code>store</code> on passed <code>row</code>, etc.
* @param store
* @param row
* @param family
* @param tr
* @return Get result.
* @throws IOException
*/
private List<Cell> doGet(final Store store, final byte [] row,
final Map.Entry<byte[], List<Cell>> family, final TimeRange tr)

View File

@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan;
/**
* Scanner scans the MOB Store. Coalesce KeyValue stream into List<KeyValue>
* Scanner scans the MOB Store. Coalesce KeyValue stream into {@code List<KeyValue>}
* for a single row. It's only used in the compaction of mob-enabled columns.
* It outputs the normal cells and delete markers when outputDeleteMarkers is set as true.
*/

View File

@ -28,9 +28,8 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.mob.MobUtils;
/**
* Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into List<KeyValue>
* for a single row.
*
* Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into
* {@code List<KeyValue>} for a single row.
*/
@InterfaceAudience.Private
public class MobStoreScanner extends StoreScanner {

View File

@ -82,14 +82,14 @@ public class ScannerContext {
* some limits and then repeatedly invoke {@link InternalScanner#next(List)} or
* {@link RegionScanner#next(List)} where each invocation respects these limits separately.
* <p>
* For example: <code><pre>
* For example: <pre> {@code
* ScannerContext context = new ScannerContext.newBuilder().setBatchLimit(5).build();
* RegionScanner scanner = ...
* List<Cell> results = new ArrayList<Cell>();
* while(scanner.next(results, context)) {
* // Do something with a batch of 5 cells
* }
* </pre></code> However, in the case of RPCs, the server wants to be able to define a set of
* }</pre> However, in the case of RPCs, the server wants to be able to define a set of
* limits for a particular RPC request and have those limits respected across multiple
* invocations. This means that the progress made towards the limits in earlier calls will be
* saved and considered in future invocations

View File

@ -361,7 +361,7 @@ public class StoreFile {
/**
* Check if this storefile was created by bulk load.
* When a hfile is bulk loaded into HBase, we append
* '_SeqId_<id-when-loaded>' to the hfile name, unless
* {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
* "hbase.mapreduce.bulkload.assign.sequenceNumbers" is
* explicitly turned off.
* If "hbase.mapreduce.bulkload.assign.sequenceNumbers"

View File

@ -68,8 +68,8 @@ public class StoreFileInfo {
Pattern.compile("^(" + DELFILE_NAME_REGEX + ")");
/**
* Regex that will work for straight reference names (<hfile>.<parentEncRegion>)
* and hfilelink reference names (<table>=<region>-<hfile>.<parentEncRegion>)
* Regex that will work for straight reference names ({@code <hfile>.<parentEncRegion>})
* and hfilelink reference names ({@code <table>=<region>-<hfile>.<parentEncRegion>})
* If reference, then the regex has more than just one group.
* Group 1, hfile/hfilelink pattern, is this file's id.
* Group 2 '(.+)' is the reference's parent region name.

View File

@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.executor.EventHandler;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext;
@ -218,9 +219,8 @@ public class OpenRegionHandler extends EventHandler {
}
/**
* Thread to run region post open tasks. Call {@link #getException()} after
* the thread finishes to check for exceptions running
* {@link RegionServerServices#postOpenDeployTasks(HRegion)
* Thread to run region post open tasks. Call {@link #getException()} after the thread finishes
* to check for exceptions running {@link RegionServerServices#postOpenDeployTasks(Region)}.
*/
static class PostOpenDeployTasksThread extends Thread {
private Throwable exception = null;

View File

@ -450,13 +450,11 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
* Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE. This
* tag type is reserved and should not be explicitly set by user.
*
* @param cell
* - the cell under consideration
* @param pair - an optional pair of type <Boolean, Tag> which would be reused
* if already set and new one will be created if null is passed
* @return a pair<Boolean, Tag> - if the boolean is false then it indicates
* that the cell has a RESERVERD_VIS_TAG and with boolean as true, not
* null tag indicates that a string modified tag was found.
* @param cell The cell under consideration
* @param pair An optional pair of type {@code <Boolean, Tag>} which would be reused if already
* set and new one will be created if NULL is passed
* @return If the boolean is false then it indicates that the cell has a RESERVERD_VIS_TAG and
* with boolean as true, not null tag indicates that a string modified tag was found.
*/
private Pair<Boolean, Tag> checkForReservedVisibilityTagPresence(Cell cell,
Pair<Boolean, Tag> pair) throws IOException {

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
/**
* Thread that walks over the filesystem, and computes the mappings
* <Region -> BestHost> and <Region -> Map<HostName, fractional-locality-of-region>>
* Region -> BestHost and Region -> {@code Map<HostName, fractional-locality-of-region>}
*
*/
@InterfaceAudience.Private

View File

@ -607,8 +607,8 @@ public class HBaseFsck extends Configured implements Closeable {
* region servers and the masters. It makes each region's state in HDFS, in
* hbase:meta, and deployments consistent.
*
* @return If &gt; 0 , number of errors detected, if &lt; 0 there was an unrecoverable
* error. If 0, we have a clean hbase.
* @return If > 0 , number of errors detected, if &lt; 0 there was an unrecoverable
* error. If 0, we have a clean hbase.
*/
public int onlineConsistencyRepair() throws IOException, KeeperException,
InterruptedException {
@ -1516,10 +1516,10 @@ public class HBaseFsck extends Configured implements Closeable {
/**
* Sideline a region dir (instead of deleting it)
*
* @param parentDir if specified, the region will be sidelined to
* folder like .../parentDir/<table name>/<region name>. The purpose
* is to group together similar regions sidelined, for example, those
* regions should be bulk loaded back later on. If null, it is ignored.
* @param parentDir if specified, the region will be sidelined to folder like
* {@literal .../parentDir/<table name>/<region name>}. The purpose is to group together
* similar regions sidelined, for example, those regions should be bulk loaded back later
* on. If NULL, it is ignored.
*/
Path sidelineRegionDir(FileSystem fs,
String parentDir, HbckInfo hi) throws IOException {

View File

@ -119,9 +119,8 @@ public class RegionMover extends AbstractHBaseTool {
private int port = HConstants.DEFAULT_REGIONSERVER_PORT;
/**
* Hostname to unload regions from or load regions to Valid format: <hostname> or
* <hostname:port>
* @param hostname
* @param hostname Hostname to unload regions from or load regions to. Can be either hostname
* or hostname:port.
*/
public RegionMoverBuilder(String hostname) {
String[] splitHostname = hostname.split(":");
@ -150,8 +149,6 @@ public class RegionMover extends AbstractHBaseTool {
/**
* Set the max number of threads that will be used to move regions
* @param threads
* @return RegionMoverBuilder object
*/
public RegionMoverBuilder maxthreads(int threads) {
this.maxthreads = threads;
@ -159,11 +156,9 @@ public class RegionMover extends AbstractHBaseTool {
}
/**
* Path of file containing hostnames to be excluded during region movement Exclude file should
* have <host:port> per line.Port is mandatory here as we can have many RS running on a single
* host
* @param excludefile
* @return RegionMoverBuilder object
* Path of file containing hostnames to be excluded during region movement. Exclude file should
* have 'host:port' per line. Port is mandatory here as we can have many RS running on a single
* host.
*/
public RegionMoverBuilder excludeFile(String excludefile) {
this.excludeFile = excludefile;
@ -751,10 +746,7 @@ public class RegionMover extends AbstractHBaseTool {
}
/**
* Create an Arraylst of servers listed in exclude file
* @param excludeFile
* @return ArrayList of servers to be excluded in format <hostname:port>
* @throws IOException
* @return List of servers from the exclude file in format 'hostname:port'.
*/
private ArrayList<String> readExcludes(String excludeFile) throws IOException {
ArrayList<String> excludeServers = new ArrayList<String>();

View File

@ -51,8 +51,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString;
// imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
@ -87,10 +85,10 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
/**
* Will block until a write entry has been assigned by they WAL subsystem.
* @return A WriteEntry gotten from local WAL subsystem. Must be completed by calling
* mvcc#complete or mvcc#completeAndWait.
* @throws InterruptedIOException
* @see
* #setWriteEntry(org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry)
* {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
* or
* {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
* @see {@link #setWriteEntry(MultiVersionConcurrencyControl.WriteEntry)}
*/
@InterfaceAudience.Private // For internal use only.
public MultiVersionConcurrencyControl.WriteEntry getWriteEntry() throws InterruptedIOException {

View File

@ -1643,7 +1643,7 @@ public class WALSplitter {
.synchronizedMap(new TreeMap<TableName, HConnection>());
/**
* Map key -> value layout
* <servername>:<table name> -> Queue<Row>
* {@literal <servername>:<table name> -> Queue<Row>}
*/
private Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
new ConcurrentHashMap<String, List<Pair<HRegionLocation, Entry>>>();
@ -2173,7 +2173,7 @@ public class WALSplitter {
}
private TableName getTableFromLocationStr(String loc) {
/**
* location key is in format <server name:port>#<table name>
* location key is in format {@literal <server name:port>#<table name>}
*/
String[] splits = loc.split(KEY_DELIMITER);
if (splits.length != 2) {