Finish up fixup of javadoc warnings

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1033624 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-11-10 18:10:32 +00:00
parent 6a5a512773
commit e4e423bd83
39 changed files with 86 additions and 104 deletions

View File

@ -439,9 +439,9 @@ public class HBaseAdmin implements Abortable {
* and {@link #isTableEnabled(byte[])} instead. * and {@link #isTableEnabled(byte[])} instead.
* @param tableName name of the table * @param tableName name of the table
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @see isTableEnabled(byte[]) * @see #isTableEnabled(byte[])
* @see disableTable(byte[]) * @see #disableTable(byte[])
* @see enableTableAsync(byte[]) * @see #enableTableAsync(byte[])
*/ */
public void enableTable(final byte [] tableName) public void enableTable(final byte [] tableName)
throws IOException { throws IOException {
@ -514,8 +514,8 @@ public class HBaseAdmin implements Abortable {
* If table is taking too long to online, check server logs. * If table is taking too long to online, check server logs.
* @param tableName name of table * @param tableName name of table
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
* @see isTableDisabled(byte[]) * @see #isTableDisabled(byte[])
* @see isTableEnabled(byte[]) * @see #isTableEnabled(byte[])
* @since 0.90.0 * @since 0.90.0
*/ */
public void disableTableAsync(final byte [] tableName) throws IOException { public void disableTableAsync(final byte [] tableName) throws IOException {

View File

@ -632,7 +632,7 @@ public class HTable implements HTableInterface {
* the {@code deletes} argument will contain the {@link Delete} instances * the {@code deletes} argument will contain the {@link Delete} instances
* that have not be successfully applied. * that have not be successfully applied.
* @since 0.20.1 * @since 0.20.1
* @see batch(java.util.List, Object[]) * @see #batch(java.util.List, Object[])
*/ */
@Override @Override
public void delete(final List<Delete> deletes) public void delete(final List<Delete> deletes)

View File

@ -317,7 +317,7 @@ public interface HTableInterface {
* Executes all the buffered {@link Put} operations. * Executes all the buffered {@link Put} operations.
* <p> * <p>
* This method gets called once automatically for every {@link Put} or batch * This method gets called once automatically for every {@link Put} or batch
* of {@link Put}s (when {@link #put(List<Put>)} is used) when * of {@link Put}s (when <code>put(List<Put>)</code> is used) when
* {@link #isAutoFlush} is {@code true}. * {@link #isAutoFlush} is {@code true}.
* @throws IOException if a remote or network exception occurs. * @throws IOException if a remote or network exception occurs.
*/ */

View File

@ -103,7 +103,7 @@ public class MetaScanner {
* *
* @param configuration HBase configuration. * @param configuration HBase configuration.
* @param visitor Visitor object. * @param visitor Visitor object.
* @param userTableName User table name in meta table to start scan at. Pass * @param tableName User table name in meta table to start scan at. Pass
* null if not interested in a particular table. * null if not interested in a particular table.
* @param row Name of the row at the user table. The scan will start from * @param row Name of the row at the user table. The scan will start from
* the region row where the row resides. * the region row where the row resides.

View File

@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.Server;
* method gets called). Implement * method gets called). Implement
* {@link EventHandlerListener}s, and registering using * {@link EventHandlerListener}s, and registering using
* {@link #setListener(EventHandlerListener)}. * {@link #setListener(EventHandlerListener)}.
* @see {@link ExecutorService} * @see ExecutorService
*/ */
public abstract class EventHandler implements Runnable, Comparable<Runnable> { public abstract class EventHandler implements Runnable, Comparable<Runnable> {
private static final Log LOG = LogFactory.getLog(EventHandler.class); private static final Log LOG = LogFactory.getLog(EventHandler.class);
@ -76,12 +76,12 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
public interface EventHandlerListener { public interface EventHandlerListener {
/** /**
* Called before any event is processed * Called before any event is processed
* @param The event handler whose process method is about to be called. * @param event The event handler whose process method is about to be called.
*/ */
public void beforeProcess(EventHandler event); public void beforeProcess(EventHandler event);
/** /**
* Called after any event is processed * Called after any event is processed
* @param The event handler whose process method is about to be called. * @param event The event handler whose process method is about to be called.
*/ */
public void afterProcess(EventHandler event); public void afterProcess(EventHandler event);
} }
@ -164,7 +164,7 @@ public abstract class EventHandler implements Runnable, Comparable<Runnable> {
/** /**
* Return the event type * Return the event type
* @return * @return The event type.
*/ */
public EventType getEventType() { public EventType getEventType() {
return this.eventType; return this.eventType;

View File

@ -47,8 +47,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
* <p>In order to use the service created above, call * <p>In order to use the service created above, call
* {@link #submit(EventHandler)}. Register pre- and post- processing listeners * {@link #submit(EventHandler)}. Register pre- and post- processing listeners
* by registering your implementation of {@link EventHandler.EventHandlerListener} * by registering your implementation of {@link EventHandler.EventHandlerListener}
* with {@link #registerListener(EventType, EventHandlerListener)}. Be sure * with {@link #registerListener(EventHandler.EventType, EventHandler.EventHandlerListener)}. Be sure
* to deregister your listener when done via {@link #unregisterListener(EventType)}. * to deregister your listener when done via {@link #unregisterListener(EventHandler.EventType)}.
*/ */
public class ExecutorService { public class ExecutorService {
private static final Log LOG = LogFactory.getLog(ExecutorService.class); private static final Log LOG = LogFactory.getLog(ExecutorService.class);
@ -151,7 +151,7 @@ public class ExecutorService {
/** /**
* Default constructor. * Default constructor.
* @param Name of the hosting server. * @param servername Name of the hosting server.
*/ */
public ExecutorService(final String servername) { public ExecutorService(final String servername) {
super(); super();
@ -226,7 +226,6 @@ public class ExecutorService {
* event type. * event type.
* @param type Type of event we're registering listener for * @param type Type of event we're registering listener for
* @param listener The listener to run. * @param listener The listener to run.
* @return The <code>listener</code> that was passed
*/ */
public void registerListener(final EventHandler.EventType type, public void registerListener(final EventHandler.EventType type,
final EventHandlerListener listener) { final EventHandlerListener listener) {

View File

@ -65,10 +65,10 @@ public class RegionTransitionData implements Writable {
* assignment. * assignment.
* *
* <p>Since only the master uses this constructor, the type should always be * <p>Since only the master uses this constructor, the type should always be
* {@link EventType#M2ZK_REGION_OFFLINE}. * {@link EventType#M_ZK_REGION_OFFLINE}.
* *
* @param eventType type of event * @param eventType type of event
* @param regionName name of region as per {@link HRegionInfo#getRegionName()} * @param regionName name of region as per <code>HRegionInfo#getRegionName()</code>
*/ */
public RegionTransitionData(EventType eventType, byte [] regionName) { public RegionTransitionData(EventType eventType, byte [] regionName) {
this(eventType, regionName, null); this(eventType, regionName, null);
@ -80,12 +80,12 @@ public class RegionTransitionData implements Writable {
* *
* <p>Used when the server name is known (a regionserver is setting it). * <p>Used when the server name is known (a regionserver is setting it).
* *
* <p>Valid types for this constructor are {@link EventType#RS2ZK_REGION_CLOSING}, * <p>Valid types for this constructor are {@link EventType#RS_ZK_REGION_CLOSING},
* {@link EventType#RS2ZK_REGION_CLOSED}, {@link EventType#RS2ZK_REGION_OPENING}, * {@link EventType#RS_ZK_REGION_CLOSED}, {@link EventType#RS_ZK_REGION_OPENING},
* and {@link EventType#RS2ZK_REGION_OPENED}. * and {@link EventType#RS_ZK_REGION_OPENED}.
* *
* @param eventType type of event * @param eventType type of event
* @param regionName name of region as per {@link HRegionInfo#getRegionName()} * @param regionName name of region as per <code>HRegionInfo#getRegionName()</code>
* @param serverName name of server setting data * @param serverName name of server setting data
*/ */
public RegionTransitionData(EventType eventType, byte [] regionName, public RegionTransitionData(EventType eventType, byte [] regionName,
@ -101,11 +101,11 @@ public class RegionTransitionData implements Writable {
* *
* <p>One of: * <p>One of:
* <ul> * <ul>
* <li>{@link EventType#M2ZK_REGION_OFFLINE} * <li>{@link EventType#M_ZK_REGION_OFFLINE}
* <li>{@link EventType#RS2ZK_REGION_CLOSING} * <li>{@link EventType#RS_ZK_REGION_CLOSING}
* <li>{@link EventType#RS2ZK_REGION_CLOSED} * <li>{@link EventType#RS_ZK_REGION_CLOSED}
* <li>{@link EventType#RS2ZK_REGION_OPENING} * <li>{@link EventType#RS_ZK_REGION_OPENING}
* <li>{@link EventType#RS2ZK_REGION_OPENED} * <li>{@link EventType#RS_ZK_REGION_OPENED}
* </ul> * </ul>
* @return type of region transition event * @return type of region transition event
*/ */
@ -117,7 +117,7 @@ public class RegionTransitionData implements Writable {
* Gets the name of the region being transitioned. * Gets the name of the region being transitioned.
* *
* <p>Region name is required so this never returns null. * <p>Region name is required so this never returns null.
* @return region name, the result of a call to {@link HRegionInfo#getRegionName()} * @return region name, the result of a call to <code>HRegionInfo#getRegionName()</code>
*/ */
public byte [] getRegionName() { public byte [] getRegionName() {
return regionName; return regionName;

View File

@ -124,10 +124,9 @@ public interface Filter extends Writable {
public void filterRow(List<KeyValue> kvs); public void filterRow(List<KeyValue> kvs);
/** /**
* Return whether or not this filter actively uses filterRow(List) * @return True if this filter actively uses filterRow(List).
* Primarily used to check for conflicts with scans(such as scans * Primarily used to check for conflicts with scans(such as scans
* that do not read a full row at a time) * that do not read a full row at a time)
* @return
*/ */
public boolean hasFilterRow(); public boolean hasFilterRow();

View File

@ -30,8 +30,6 @@ import java.util.List;
* *
* If you could instantiate FilterBase, it would end up being a "null" filter - * If you could instantiate FilterBase, it would end up being a "null" filter -
* that is one that never filters anything. * that is one that never filters anything.
*
* @inheritDoc
*/ */
public abstract class FilterBase implements Filter { public abstract class FilterBase implements Filter {
@ -101,7 +99,7 @@ public abstract class FilterBase implements Filter {
/** /**
* Filters that never filter by rows based on previously gathered state from * Filters that never filter by rows based on previously gathered state from
* @{link #filterKeyValue(KeyValue)} can inherit this implementation that * {@link #filterKeyValue(KeyValue)} can inherit this implementation that
* never filters a row. * never filters a row.
* *
* @inheritDoc * @inheritDoc

View File

@ -13,9 +13,9 @@ import org.apache.hadoop.hbase.KeyValue;
* in the specified list of timestamps (versions). * in the specified list of timestamps (versions).
* <p> * <p>
* Note: Use of this filter overrides any time range/time stamp * Note: Use of this filter overrides any time range/time stamp
* options specified using {@link Get#setTimeRange(long, long)}, * options specified using {@link org.apache.hadoop.hbase.client.Get#setTimeRange(long, long)},
* {@link Scan#setTimeRange(long, long)}, {@link Get#setTimeStamp(long)}, * {@link org.apache.hadoop.hbase.client.Scan#setTimeRange(long, long)}, {@link org.apache.hadoop.hbase.client.Get#setTimeStamp(long)},
* or {@link Scan#setTimeStamp(long)}. * or {@link org.apache.hadoop.hbase.client.Scan#setTimeStamp(long)}.
*/ */
public class TimestampsFilter extends FilterBase { public class TimestampsFilter extends FilterBase {

View File

@ -51,7 +51,7 @@ public class ByteBufferOutputStream extends OutputStream {
/** /**
* This flips the underlying BB so be sure to use it _last_! * This flips the underlying BB so be sure to use it _last_!
* @return * @return ByteBuffer
*/ */
public ByteBuffer getByteBuffer() { public ByteBuffer getByteBuffer() {
buf.flip(); buf.flip();

View File

@ -312,16 +312,14 @@ public interface HRegionInterface extends HBaseRPCProtocolVersion, Stoppable, Ab
/** /**
* Opens the specified region. * Opens the specified region.
* @param region region to open * @param region region to open
* @param RegionServerStoppedException * @throws IOException
* @param IOException
*/ */
public void openRegion(final HRegionInfo region) throws IOException; public void openRegion(final HRegionInfo region) throws IOException;
/** /**
* Opens the specified regions. * Opens the specified regions.
* @param regions regions to open * @param regions regions to open
* @param RegionServerStoppedException * @throws IOException
* @param IOException
*/ */
public void openRegions(final List<HRegionInfo> regions) throws IOException; public void openRegions(final List<HRegionInfo> regions) throws IOException;

View File

@ -75,7 +75,7 @@ Writing MapReduce jobs that read or write HBase, you'll probably want to subclas
{@link org.apache.hadoop.hbase.mapred.TableReduce TableReduce}. See the do-nothing {@link org.apache.hadoop.hbase.mapred.TableReduce TableReduce}. See the do-nothing
pass-through classes {@link org.apache.hadoop.hbase.mapred.IdentityTableMap IdentityTableMap} and pass-through classes {@link org.apache.hadoop.hbase.mapred.IdentityTableMap IdentityTableMap} and
{@link org.apache.hadoop.hbase.mapred.IdentityTableReduce IdentityTableReduce} for basic usage. For a more {@link org.apache.hadoop.hbase.mapred.IdentityTableReduce IdentityTableReduce} for basic usage. For a more
involved example, see {@link org.apache.hadoop.hbase.mapred.BuildTableIndex BuildTableIndex} involved example, see <code>BuildTableIndex</code>
or review the <code>org.apache.hadoop.hbase.mapred.TestTableMapReduce</code> unit test. or review the <code>org.apache.hadoop.hbase.mapred.TestTableMapReduce</code> unit test.
</p> </p>

View File

@ -54,7 +54,7 @@ import org.apache.hadoop.util.ToolRunner;
/** /**
* Tool to load the output of HFileOutputFormat into an existing table. * Tool to load the output of HFileOutputFormat into an existing table.
* @see usage() * @see #usage()
*/ */
public class LoadIncrementalHFiles extends Configured implements Tool { public class LoadIncrementalHFiles extends Configured implements Tool {

View File

@ -57,7 +57,7 @@ implements Configurable {
* Optional job parameter to specify a peer cluster. * Optional job parameter to specify a peer cluster.
* Used specifying remote cluster when copying between hbase clusters (the * Used specifying remote cluster when copying between hbase clusters (the
* source is picked up from <code>hbase-site.xml</code>). * source is picked up from <code>hbase-site.xml</code>).
* @see {@link TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, Class, String, String, String)} * @see TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, Class, String, String, String)
*/ */
public static final String QUORUM_ADDRESS = "hbase.mapred.output.quorum"; public static final String QUORUM_ADDRESS = "hbase.mapred.output.quorum";

View File

@ -45,7 +45,7 @@ the hadoop cluster before it'll notice your HBase additions.</p>
<p>As of 0.90.x, HBase will just add its dependency jars to the job <p>As of 0.90.x, HBase will just add its dependency jars to the job
configuration; the dependencies just need to be available on the local configuration; the dependencies just need to be available on the local
<code>CLASSPATH</code>. For example, to run the bundled HBase <code>CLASSPATH</code>. For example, to run the bundled HBase
{@link RowCounter} mapreduce job against a table named <code>usertable</code>, {@link org.apache.hadoop.hbase.mapreduce.RowCounter} mapreduce job against a table named <code>usertable</code>,
type: type:
<blockquote><pre> <blockquote><pre>

View File

@ -136,9 +136,7 @@ public class AssignmentManager extends ZooKeeperListener {
/** /**
* Constructs a new assignment manager. * Constructs a new assignment manager.
* *
* <p>This manager must be started with {@link #start()}. * @param master
*
* @param status master status
* @param serverManager * @param serverManager
* @param catalogTracker * @param catalogTracker
* @param service * @param service
@ -574,7 +572,6 @@ public class AssignmentManager extends ZooKeeperListener {
* <p> * <p>
* Used when a region has been closed and should remain closed. * Used when a region has been closed and should remain closed.
* @param regionInfo * @param regionInfo
* @param serverInfo
*/ */
public void regionOffline(final HRegionInfo regionInfo) { public void regionOffline(final HRegionInfo regionInfo) {
synchronized(this.regionsInTransition) { synchronized(this.regionsInTransition) {
@ -638,7 +635,7 @@ public class AssignmentManager extends ZooKeeperListener {
* chosen server is up and running (It may have just crashed!). If the * chosen server is up and running (It may have just crashed!). If the
* in-memory checks pass, the zk node is forced to OFFLINE before assigning. * in-memory checks pass, the zk node is forced to OFFLINE before assigning.
* *
* @param regionName server to be assigned * @param region server to be assigned
* @param setOfflineInZK whether ZK node should be created/transitioned to an * @param setOfflineInZK whether ZK node should be created/transitioned to an
* OFFLINE state before assigning the region * OFFLINE state before assigning the region
*/ */
@ -956,11 +953,9 @@ public class AssignmentManager extends ZooKeeperListener {
* <p> * <p>
* Updates the RegionState and sends the CLOSE RPC. * Updates the RegionState and sends the CLOSE RPC.
* <p> * <p>
* If a RegionPlan is already set, it will remain. If this is being used * If a RegionPlan is already set, it will remain.
* to disable a table, be sure to use {@link #setDisabledTable(String)} to ensure
* regions are not onlined after being closed.
* *
* @param regionName server to be unassigned * @param region server to be unassigned
*/ */
public void unassign(HRegionInfo region) { public void unassign(HRegionInfo region) {
unassign(region, false); unassign(region, false);
@ -971,11 +966,9 @@ public class AssignmentManager extends ZooKeeperListener {
* <p> * <p>
* Updates the RegionState and sends the CLOSE RPC. * Updates the RegionState and sends the CLOSE RPC.
* <p> * <p>
* If a RegionPlan is already set, it will remain. If this is being used * If a RegionPlan is already set, it will remain.
* to disable a table, be sure to use {@link #setDisabledTable(String)} to ensure
* regions are not onlined after being closed.
* *
* @param regionName server to be unassigned * @param region server to be unassigned
* @param force if region should be closed even if already closing * @param force if region should be closed even if already closing
*/ */
public void unassign(HRegionInfo region, boolean force) { public void unassign(HRegionInfo region, boolean force) {

View File

@ -37,7 +37,6 @@ public abstract class BulkAssigner {
/** /**
* @param server An instance of Server * @param server An instance of Server
* @param regionsInTransition A reference to {@link AssignmentManager#regionsInTransition}
*/ */
public BulkAssigner(final Server server) { public BulkAssigner(final Server server) {
this.server = server; this.server = server;

View File

@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.HServerInfo;
* the Master needs to handle closed regions that it currently does not have * the Master needs to handle closed regions that it currently does not have
* a destination set for. This can happen during master failover. * a destination set for. This can happen during master failover.
* *
* <p>On cluster startup, {@link #bulkAssignment} can be used to determine * <p>On cluster startup, bulk assignment can be used to determine
* locations for all Regions in a cluster. * locations for all Regions in a cluster.
* *
* <p>This classes produces plans for the {@link AssignmentManager} to execute. * <p>This classes produces plans for the {@link AssignmentManager} to execute.
@ -122,7 +122,7 @@ public class LoadBalancer {
* Does this mean we need HeapSize on HMaster? Or just careful monitor? * Does this mean we need HeapSize on HMaster? Or just careful monitor?
* (current thinking is we will hold all assignments in memory) * (current thinking is we will hold all assignments in memory)
* *
* @param serverInfo map of regionservers and their load/region information to * @param clusterState Map of regionservers and their load/region information to
* a list of their most loaded regions * a list of their most loaded regions
* @return a list of regions to be moved, including source and destination, * @return a list of regions to be moved, including source and destination,
* or null if cluster is already balanced * or null if cluster is already balanced

View File

@ -96,8 +96,6 @@ public class ServerManager {
* @param master * @param master
* @param services * @param services
* @param metrics * @param metrics
* @param freshClusterStartup True if we are original master on a fresh
* cluster startup else if false, we are joining an already running cluster.
*/ */
public ServerManager(final Server master, final MasterServices services, public ServerManager(final Server master, final MasterServices services,
MasterMetrics metrics) { MasterMetrics metrics) {
@ -564,7 +562,7 @@ public class ServerManager {
* A region server could reject the close request because it either does not * A region server could reject the close request because it either does not
* have the specified region or the region is being split. * have the specified region or the region is being split.
* @param server server to open a region * @param server server to open a region
* @param regionName region to open * @param region region to open
* @return true if server acknowledged close, false if not * @return true if server acknowledged close, false if not
* @throws IOException * @throws IOException
*/ */

View File

@ -154,13 +154,11 @@ public class ServerShutdownHandler extends EventHandler {
/** /**
* Process a dead region from a dead RS. Checks if the region is disabled * Process a dead region from a dead RS. Checks if the region is disabled
* or if the region has a partially completed split. * or if the region has a partially completed split.
* <p>
* Returns true if specified region should be assigned, false if not.
* @param hri * @param hri
* @param result * @param result
* @param assignmentManager * @param assignmentManager
* @param catalogTracker * @param catalogTracker
* @return * @return Returns true if specified region should be assigned, false if not.
* @throws IOException * @throws IOException
*/ */
public static boolean processDeadRegion(HRegionInfo hri, Result result, public static boolean processDeadRegion(HRegionInfo hri, Result result,

View File

@ -885,7 +885,7 @@ public class HRegion implements HeapSize { // , Writable{
* flush file. * flush file.
* @return true if the region needs compacting * @return true if the region needs compacting
* @throws IOException * @throws IOException
* @see {@link #internalFlushcache()} * @see #internalFlushcache()
*/ */
protected boolean internalFlushcache(final HLog wal, final long myseqid) protected boolean internalFlushcache(final HLog wal, final long myseqid)
throws IOException { throws IOException {
@ -2435,7 +2435,6 @@ public class HRegion implements HeapSize { // , Writable{
/** /**
* Open a Region. * Open a Region.
* @param info Info for region to be opened. * @param info Info for region to be opened.
* @param rootDir Root directory for HBase instance
* @param wal HLog for region to use. This method will call * @param wal HLog for region to use. This method will call
* HLog#setSequenceNumber(long) passing the result of the call to * HLog#setSequenceNumber(long) passing the result of the call to
* HRegion#getMinSequenceId() to ensure the log id is properly kept * HRegion#getMinSequenceId() to ensure the log id is properly kept

View File

@ -2017,7 +2017,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
* @param zk True if we are to update zk about the region close; if the close * @param zk True if we are to update zk about the region close; if the close
* was orchestrated by master, then update zk. If the close is being run by * was orchestrated by master, then update zk. If the close is being run by
* the regionserver because its going down, don't update zk. * the regionserver because its going down, don't update zk.
* @return * @return True if closed a region.
*/ */
protected boolean closeRegion(HRegionInfo region, final boolean abort, protected boolean closeRegion(HRegionInfo region, final boolean abort,
final boolean zk) { final boolean zk) {

View File

@ -360,7 +360,7 @@ public class MemStore implements HeapSize {
* @param qualifier * @param qualifier
* @param newValue * @param newValue
* @param now * @param now
* @return * @return Timestamp
*/ */
public long updateColumnValue(byte[] row, public long updateColumnValue(byte[] row,
byte[] family, byte[] family,

View File

@ -160,7 +160,7 @@ public class PriorityCompactionQueue implements BlockingQueue<HRegion> {
} }
/** Removes the request from the regions in queue /** Removes the request from the regions in queue
* @param p If null it will use the default priority * @param remove
*/ */
protected CompactionRequest removeFromRegionsInQueue(CompactionRequest remove) { protected CompactionRequest removeFromRegionsInQueue(CompactionRequest remove) {
if (remove == null) return null; if (remove == null) return null;

View File

@ -434,6 +434,7 @@ public class Store implements HeapSize {
* previously. * previously.
* @param logCacheFlushId flush sequence number * @param logCacheFlushId flush sequence number
* @param snapshot * @param snapshot
* @param snapshotTimeRangeTracker
* @return true if a compaction is needed * @return true if a compaction is needed
* @throws IOException * @throws IOException
*/ */

View File

@ -443,7 +443,7 @@ public class StoreFile {
/** /**
* @return Current reader. Must call createReader first else returns null. * @return Current reader. Must call createReader first else returns null.
* @throws IOException * @throws IOException
* @see {@link #createReader()} * @see #createReader()
*/ */
public Reader getReader() { public Reader getReader() {
return this.reader; return this.reader;

View File

@ -292,7 +292,7 @@ public class HLog implements Syncable {
* @param conf configuration to use * @param conf configuration to use
* @param listeners Listeners on WAL events. Listeners passed here will * @param listeners Listeners on WAL events. Listeners passed here will
* be registered before we do anything else; e.g. the * be registered before we do anything else; e.g. the
* Constructor {@link #rollWriter(). * Constructor {@link #rollWriter()}.
* @param prefix should always be hostname and port in distributed env and * @param prefix should always be hostname and port in distributed env and
* it will be URL encoded before being used. * it will be URL encoded before being used.
* If prefix is null, "hlog" will be used * If prefix is null, "hlog" will be used
@ -317,7 +317,7 @@ public class HLog implements Syncable {
* @param conf configuration to use * @param conf configuration to use
* @param listeners Listeners on WAL events. Listeners passed here will * @param listeners Listeners on WAL events. Listeners passed here will
* be registered before we do anything else; e.g. the * be registered before we do anything else; e.g. the
* Constructor {@link #rollWriter(). * Constructor {@link #rollWriter()}.
* @param failIfLogDirExists If true IOException will be thrown if dir already exists. * @param failIfLogDirExists If true IOException will be thrown if dir already exists.
* @param prefix should always be hostname and port in distributed env and * @param prefix should always be hostname and port in distributed env and
* it will be URL encoded before being used. * it will be URL encoded before being used.
@ -541,7 +541,7 @@ public class HLog implements Syncable {
* @param fs * @param fs
* @param path * @param path
* @param conf * @param conf
* @return * @return Writer instance
* @throws IOException * @throws IOException
*/ */
protected Writer createWriterInstance(final FileSystem fs, final Path path, protected Writer createWriterInstance(final FileSystem fs, final Path path,
@ -752,7 +752,7 @@ public class HLog implements Syncable {
/** /**
* This is a convenience method that computes a new filename with a given * This is a convenience method that computes a new filename with a given
* file-number. * file-number.
* @param file-number to use * @param filenum to use
* @return Path * @return Path
*/ */
protected Path computeFilename(long filenum) { protected Path computeFilename(long filenum) {

View File

@ -59,7 +59,7 @@ public class HLogKey implements WritableComparable<HLogKey> {
* A regionName is always a sub-table object. * A regionName is always a sub-table object.
* *
* @param encodedRegionName Encoded name of the region as returned by * @param encodedRegionName Encoded name of the region as returned by
* {@link HRegionInfo#getEncodedNameAsBytes()}. * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
* @param tablename - name of table * @param tablename - name of table
* @param logSeqNum - log sequence number * @param logSeqNum - log sequence number
* @param now Time at which this edit was written. * @param now Time at which this edit was written.
@ -101,7 +101,7 @@ public class HLogKey implements WritableComparable<HLogKey> {
/** /**
* Get the id of the original cluster * Get the id of the original cluster
* @return * @return Cluster id.
*/ */
public byte getClusterId() { public byte getClusterId() {
return clusterId; return clusterId;

View File

@ -491,7 +491,7 @@ public class ReplicationSource extends Thread
* Do the sleeping logic * Do the sleeping logic
* @param msg Why we sleep * @param msg Why we sleep
* @param sleepMultiplier by how many times the default sleeping time is augmented * @param sleepMultiplier by how many times the default sleeping time is augmented
* @return * @return True if <code>sleepMultiplier</code> is &lt; <code>maxRetriesMultiplier</code>
*/ */
protected boolean sleepForRetries(String msg, int sleepMultiplier) { protected boolean sleepForRetries(String msg, int sleepMultiplier) {
try { try {

View File

@ -521,7 +521,7 @@ public class ReplicationSourceManager {
/** /**
* Get the handle on the local file system * Get the handle on the local file system
* @returnthe handle on the local file system * @return Handle on the local file system
*/ */
public FileSystem getFs() { public FileSystem getFs() {
return this.fs; return this.fs;

View File

@ -36,7 +36,7 @@ public interface Transform {
/** /**
* Transform data from one representation to another according to * Transform data from one representation to another according to
* transfer direction. * transfer direction.
* @param in input data * @param data input data
* @param direction IN or OUT * @param direction IN or OUT
* @return the transformed data * @return the transformed data
*/ */

View File

@ -65,7 +65,7 @@ public abstract class User {
/** /**
* Returns the full user name. For Kerberos principals this will include * Returns the full user name. For Kerberos principals this will include
* the host and realm portions of the principal name. * the host and realm portions of the principal name.
* @return * @return User full name.
*/ */
public String getName() { public String getName() {
return ugi.getUserName(); return ugi.getUserName();

View File

@ -98,7 +98,8 @@ public class DynamicByteBloomFilter implements BloomFilter {
/** /**
* Normal write constructor. Note that this doesn't allocate bloom data by * Normal write constructor. Note that this doesn't allocate bloom data by
* default. Instead, call allocBloom() before adding entries. * default. Instead, call allocBloom() before adding entries.
* @param hashType type of the hashing function (see {@link org.apache.hadoop.util.hash.Hash}). * @param errorRate
* @param hashType type of the hashing function (see <code>org.apache.hadoop.util.hash.Hash</code>).
* @param keyInterval Maximum number of keys to record per Bloom filter row. * @param keyInterval Maximum number of keys to record per Bloom filter row.
* @throws IllegalArgumentException The input parameters were invalid * @throws IllegalArgumentException The input parameters were invalid
*/ */

View File

@ -166,8 +166,8 @@ public class JVMClusterUtil {
/** /**
* Start the cluster. Waits until there is a primary master and returns its * Start the cluster. Waits until there is a primary master and returns its
* address. * address.
* @param m * @param masters
* @param regionServers * @param regionservers
* @return Address to use contacting primary master. * @return Address to use contacting primary master.
*/ */
public static String startup(final List<JVMClusterUtil.MasterThread> masters, public static String startup(final List<JVMClusterUtil.MasterThread> masters,
@ -201,7 +201,7 @@ public class JVMClusterUtil {
} }
/** /**
* @param master * @param masters
* @param regionservers * @param regionservers
*/ */
public static void shutdown(final List<MasterThread> masters, public static void shutdown(final List<MasterThread> masters,

View File

@ -27,7 +27,7 @@ import org.apache.commons.lang.NotImplementedException;
/** /**
* A generic, immutable class for pairs of objects both of type <code>T</code>. * A generic, immutable class for pairs of objects both of type <code>T</code>.
* @param <T> * @param <T>
* @See {@link Pair} if Types differ. * @see Pair if Types differ.
*/ */
public class PairOfSameType<T> implements Iterable<T> { public class PairOfSameType<T> implements Iterable<T> {
private final T first; private final T first;

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/** /**
* Tracks the root region server location node in zookeeper. * Tracks the root region server location node in zookeeper.
* Root region location is set by {@link RootLocationEditor} usually called * Root region location is set by {@link RootLocationEditor} usually called
* out of {@link RegionServerServices#postOpenDeployTasks(org.apache.hadoop.hbase.regionserver.HRegion, org.apache.hadoop.hbase.catalog.CatalogTracker)}. * out of <code>RegionServerServices</code>.
* This class has a watcher on the root location and notices changes. * This class has a watcher on the root location and notices changes.
*/ */
public class RootRegionTracker extends ZooKeeperNodeTracker { public class RootRegionTracker extends ZooKeeperNodeTracker {
@ -64,8 +64,7 @@ public class RootRegionTracker extends ZooKeeperNodeTracker {
/** /**
* Gets the root region location, if available, and waits for up to the * Gets the root region location, if available, and waits for up to the
* specified timeout if not immediately available. * specified timeout if not immediately available.
* @param timeout maximum time to wait, in millis, use {@link ZooKeeperNodeTracker#NO_TIMEOUT} for * @param timeout maximum time to wait, in millis
* forever
* @return server address for server hosting root region, null if timed out * @return server address for server hosting root region, null if timed out
* @throws InterruptedException if interrupted while waiting * @throws InterruptedException if interrupted while waiting
*/ */

View File

@ -243,7 +243,7 @@ public class ZKConfig {
/** /**
* Return the ZK Quorum servers string given the specified configuration. * Return the ZK Quorum servers string given the specified configuration.
* @param properties * @param conf
* @return Quorum servers * @return Quorum servers
*/ */
public static String getZKQuorumServersString(Configuration conf) { public static String getZKQuorumServersString(Configuration conf) {

View File

@ -63,8 +63,8 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
/** /**
* Starts the tracking of the node in ZooKeeper. * Starts the tracking of the node in ZooKeeper.
* *
* <p>Use {@link blockUntilAvailable()} to block until the node is available * <p>Use {@link #blockUntilAvailable()} to block until the node is available
* or {@link getData()} to get the data of the node if it is available. * or {@link #getData()} to get the data of the node if it is available.
*/ */
public synchronized void start() { public synchronized void start() {
this.watcher.registerListener(this); this.watcher.registerListener(this);